Skip to content

Commit

Permalink
Add special handler to allow logger messages during shutdown
Browse files Browse the repository at this point in the history
* Adds a special PropertyChecker interface which is ONLY for setting string properties at the very start of psvm
  • Loading branch information
drcrallen committed Oct 27, 2015
1 parent 4f746dd commit 7a2ceef
Show file tree
Hide file tree
Showing 8 changed files with 361 additions and 1 deletion.
141 changes: 141 additions & 0 deletions common/src/main/java/io/druid/common/config/Log4jShutdown.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.common.config;

import org.apache.logging.log4j.core.util.Cancellable;
import org.apache.logging.log4j.core.util.ShutdownCallbackRegistry;

import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;

public class Log4jShutdown implements ShutdownCallbackRegistry, org.apache.logging.log4j.core.LifeCycle
{
private final AtomicReference<State> state = new AtomicReference<>(State.INITIALIZED);
private final Queue<Runnable> shutdownCallbacks = new ConcurrentLinkedQueue<>();
private final AtomicBoolean callbacksRun = new AtomicBoolean(false);

@Override
public Cancellable addShutdownCallback(final Runnable callback)
{
if (callback == null) {
throw new NullPointerException("callback");
}
if (!isStarted()) {
throw new IllegalStateException("Not started");
}
final Cancellable cancellable = new Cancellable()
{
private volatile boolean cancelled = false;
private final AtomicBoolean ran = new AtomicBoolean(false);

@Override
public void cancel()
{
cancelled = true;
}

@Override
public void run()
{
if (!cancelled) {
if (ran.compareAndSet(false, true)) {
callback.run();
}
}
}
};
shutdownCallbacks.add(cancellable);
if (!isStarted()) {
// We are shutting down in the middle of registering... Make sure the callback fires
callback.run();
throw new IllegalStateException("Shutting down while adding shutdown hook. Callback fired just in case");
}
return cancellable;
}

@Override
public State getState()
{
return state.get();
}

@Override
public void initialize()
{
// NOOP, state is always at least INITIALIZED
}

@Override
public void start()
{
if (!state.compareAndSet(State.INITIALIZED, State.STARTED)) { // Skip STARTING
throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.INITIALIZED, state.get()));
}
}

@Override
public void stop()
{
if (callbacksRun.get()) {
return;
}
if (!state.compareAndSet(State.STARTED, State.STOPPED)) {
throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.STARTED, state.get()));
}
}

public void runCallbacks()
{
if (!callbacksRun.compareAndSet(false, true)) {
// Already run, skip
return;
}
stop();
RuntimeException e = null;
for (Runnable callback = shutdownCallbacks.poll(); callback != null; callback = shutdownCallbacks.poll()) {
try {
callback.run();
}
catch (RuntimeException ex) {
if (e == null) {
e = new RuntimeException("Error running callback");
}
e.addSuppressed(ex);
}
}
if (e != null) {
throw e;
}
}

@Override
public boolean isStarted()
{
return State.STARTED.equals(getState());
}

@Override
public boolean isStopped()
{
return State.STOPPED.equals(getState());
}
}
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@
<druid.api.version>0.3.13</druid.api.version>
<!-- Watch out for Hadoop compatibility when updating to >= 2.5; see https://github.com/druid-io/druid/pull/1669 -->
<jackson.version>2.4.6</jackson.version>
<log4j.version>2.3</log4j.version>
<log4j.version>2.4.1</log4j.version>
<slf4j.version>1.7.12</slf4j.version>
<hadoop.compile.version>2.3.0</hadoop.compile.version>
</properties>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter
{
final ModuleList defaultModules = new ModuleList(baseInjector);
defaultModules.addModules(
new Log4jShutterDownerModule(),
new LifecycleModule(),
EmitterModule.class,
HttpClientModule.global(),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.initialization;

import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.Module;
import com.google.inject.Provides;
import com.google.inject.name.Names;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import io.druid.common.config.Log4jShutdown;
import io.druid.guice.ManageLifecycle;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.core.impl.Log4jContextFactory;
import org.apache.logging.log4j.core.util.ShutdownCallbackRegistry;
import org.apache.logging.log4j.spi.LoggerContextFactory;

public class Log4jShutterDownerModule implements Module
{
private static final Logger log = new Logger(Log4jShutterDownerModule.class);

@Override
public void configure(Binder binder)
{
// Instantiate eagerly so that we get everything registered and put into the Lifecycle
// This makes the shutdown run pretty darn near last.

try {
// Reflection to try and allow non Log4j2 stuff to run. This acts as a gateway to stop errors in the next few lines
final Class<?> logManagerClazz = Class.forName("org.apache.logging.log4j.LogManager");

final LoggerContextFactory contextFactory = LogManager.getFactory();
if (!(contextFactory instanceof Log4jContextFactory)) {
log.warn(
"Expected [%s] found [%s]. Unknown class for context factory. Not logging shutdown",
Log4jContextFactory.class.getCanonicalName(),
contextFactory.getClass().getCanonicalName()
);
return;
}
final ShutdownCallbackRegistry registry = ((Log4jContextFactory) contextFactory).getShutdownCallbackRegistry();
if (!(registry instanceof Log4jShutdown)) {
log.warn(
"Shutdown callback registry expected class [%s] found [%s]. Skipping shutdown registry",
Log4jShutdown.class.getCanonicalName(),
registry.getClass().getCanonicalName()
);
return;
}
binder.bind(Log4jShutdown.class).toInstance((Log4jShutdown) registry);
binder.bind(Key.get(Log4jShutterDowner.class, Names.named("ForTheEagerness")))
.to(Log4jShutterDowner.class)
.asEagerSingleton();
}
catch (ClassNotFoundException | ClassCastException | LinkageError e) {
log.warn(e, "Not registering log4j shutdown hooks. Not using log4j?");
}
}


@ManageLifecycle
@Provides
public Log4jShutterDowner getShutterDowner(
Log4jShutdown log4jShutdown
)
{
return new Log4jShutterDowner(log4jShutdown);
}

public static class Log4jShutterDowner
{
private final Log4jShutdown log4jShutdown;

public Log4jShutterDowner(Log4jShutdown log4jShutdown)
{
this.log4jShutdown = log4jShutdown;
}

@LifecycleStart
public void start()
{
log.debug("Log4j shutter downer is waiting");
}

@LifecycleStop
public void stop()
{
if (log4jShutdown != null) {
log.debug("Shutting down log4j");
log4jShutdown.stop();
} else {
log.warn("Log4j shutdown was registered in lifecycle but no shutdown object exists!");
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.cli;

import java.util.Properties;

public class Log4JShutdownPropertyChecker implements PropertyChecker
{
@Override
public void checkProperties(Properties properties)
{
if (!properties.contains("log4j.shutdownCallbackRegistry")) {
properties.setProperty("log4j.shutdownCallbackRegistry", "io.druid.common.config.Log4jShutdown");
}
if (!properties.contains("log4j.shutdownHookEnabled")) {
properties.setProperty("log4j.shutdownHookEnabled", "true");
}
}
}
8 changes: 8 additions & 0 deletions services/src/main/java/io/druid/cli/Main.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,11 +28,19 @@
import io.druid.initialization.Initialization;

import java.util.Collection;
import java.util.ServiceLoader;

/**
*/
public class Main
{
static {
ServiceLoader<PropertyChecker> serviceLoader = ServiceLoader.load(PropertyChecker.class);
for (PropertyChecker propertyChecker : serviceLoader) {
propertyChecker.checkProperties(System.getProperties());
}
}

@SuppressWarnings("unchecked")
public static void main(String[] args)
{
Expand Down
39 changes: 39 additions & 0 deletions services/src/main/java/io/druid/cli/PropertyChecker.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package io.druid.cli;

import java.util.Properties;

/**
* The PropertyChecker classes are loaded by ServiceLoader at the very start of the program and as such MUST be on the
* initial classpath and cannot be loaded via extensions at runtime. (Or more precisely, they are ignored if present
* in an extension at runtime, but not on the initial classpath)
*
* The PropertyChecker should ONLY try and set very specific properties and any class loading should be done in an
* isolated class loader to not pollute the general class loader
*/
public interface PropertyChecker
{
/**
* Check the given properties to make sure any unset values are properly configured.
* @param properties The properties to check, usually System.getProperties()
*/
void checkProperties(Properties properties);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
#
# Licensed to Metamarkets Group Inc. (Metamarkets) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. Metamarkets licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#

io.druid.cli.Log4JShutdownPropertyChecker

0 comments on commit 7a2ceef

Please sign in to comment.