You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by fj...@apache.org on 2019/01/21 17:01:43 UTC

[incubator-druid] branch master updated: add 'init' lifecycle stage for finer control over startup and shutdown (#6864)

This is an automated email from the ASF dual-hosted git repository.

fjy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new 8ba33b2  add 'init' lifecycle stage for finer control over startup and shutdown (#6864)
8ba33b2 is described below

commit 8ba33b250572fc7f4a1b3b2632f5c18f8c562b9a
Author: Clint Wylie <cj...@gmail.com>
AuthorDate: Mon Jan 21 09:01:36 2019 -0800

    add 'init' lifecycle stage for finer control over startup and shutdown (#6864)
    
    * add Lifecycle.Stage.INIT, put log shutter downer in init stage, tests, rad startup banner
    
    * log cleanup
    
    * log changes
    
    * add task-master lifecycle to module lifecycle to gracefully stop task-master stuff
    
    * fix it the right way
    
    * remove announce spam
    
    * unused import
    
    * one more log
    
    * updated comments
    
    * wrap leadership lifecycle stop to prevent exceptions from wrecking rest of task master stop
    
    * add precondition check
---
 .../org/apache/druid/guice/LifecycleModule.java    |  7 +++-
 .../apache/druid/guice/ManageLifecycleInit.java    | 47 +++++++++-------------
 .../java/util/common/lifecycle/Lifecycle.java      | 31 ++++++++++----
 .../java/util/common/lifecycle/LifecycleTest.java  | 14 ++++---
 .../apache/druid/indexing/overlord/TaskMaster.java | 16 +++++++-
 .../apache/druid/guice/http/LifecycleUtils.java    |  2 +-
 .../initialization/Log4jShutterDownerModule.java   |  8 ++--
 7 files changed, 75 insertions(+), 50 deletions(-)

diff --git a/core/src/main/java/org/apache/druid/guice/LifecycleModule.java b/core/src/main/java/org/apache/druid/guice/LifecycleModule.java
index bbf9be4..bbe5ccf 100644
--- a/core/src/main/java/org/apache/druid/guice/LifecycleModule.java
+++ b/core/src/main/java/org/apache/druid/guice/LifecycleModule.java
@@ -37,6 +37,9 @@ import java.util.Set;
  */
 public class LifecycleModule implements Module
 {
+  // this scope includes final logging shutdown, so all other handlers in this lifecycle scope should avoid logging in
+  // the 'stop' method, either failing silently or failing violently and throwing an exception causing an ungraceful exit
+  private final LifecycleScope initScope = new LifecycleScope(Lifecycle.Stage.INIT);
   private final LifecycleScope scope = new LifecycleScope(Lifecycle.Stage.NORMAL);
   private final LifecycleScope lastScope = new LifecycleScope(Lifecycle.Stage.LAST);
 
@@ -113,6 +116,7 @@ public class LifecycleModule implements Module
   {
     getEagerBinder(binder); // Load up the eager binder so that it will inject the empty set at a minimum.
 
+    binder.bindScope(ManageLifecycleInit.class, initScope);
     binder.bindScope(ManageLifecycle.class, scope);
     binder.bindScope(ManageLifecycleLast.class, lastScope);
   }
@@ -123,7 +127,7 @@ public class LifecycleModule implements Module
     final Key<Set<KeyHolder>> keyHolderKey = Key.get(new TypeLiteral<Set<KeyHolder>>(){}, Names.named("lifecycle"));
     final Set<KeyHolder> eagerClasses = injector.getInstance(keyHolderKey);
 
-    Lifecycle lifecycle = new Lifecycle()
+    Lifecycle lifecycle = new Lifecycle("module")
     {
       @Override
       public void start() throws Exception
@@ -134,6 +138,7 @@ public class LifecycleModule implements Module
         super.start();
       }
     };
+    initScope.setLifecycle(lifecycle);
     scope.setLifecycle(lifecycle);
     lastScope.setLifecycle(lifecycle);
 
diff --git a/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java b/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java
similarity index 53%
copy from server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java
copy to core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java
index 4140eaf..f8a3750 100644
--- a/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java
+++ b/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java
@@ -17,36 +17,25 @@
  * under the License.
  */
 
-package org.apache.druid.guice.http;
+package org.apache.druid.guice;
 
-import com.google.common.base.Throwables;
-import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import com.google.inject.ScopeAnnotation;
+import org.apache.druid.guice.annotations.PublicApi;
 
-public class LifecycleUtils
-{
-  public static Lifecycle asMmxLifecycle(Lifecycle lifecycle)
-  {
-    final Lifecycle metamxLifecycle = new Lifecycle();
-    try {
-      lifecycle.addMaybeStartHandler(new Lifecycle.Handler()
-      {
-        @Override
-        public void start() throws Exception
-        {
-          metamxLifecycle.start();
-        }
-
-        @Override
-        public void stop()
-        {
-          metamxLifecycle.stop();
-        }
-      });
-    }
-    catch (Exception e) {
-      throw Throwables.propagate(e);
-    }
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
-    return metamxLifecycle;
-  }
+/**
+ * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on Stage.INIT
+ *
+ * This Scope gets defined by {@link LifecycleModule}
+ */
+@Target({ ElementType.TYPE, ElementType.METHOD })
+@Retention(RetentionPolicy.RUNTIME)
+@ScopeAnnotation
+@PublicApi
+public @interface ManageLifecycleInit
+{
 }
diff --git a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java
index 3cb76fa..d26736c 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java
@@ -19,7 +19,9 @@
 
 package org.apache.druid.java.util.common.lifecycle;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.logger.Logger;
 
@@ -40,10 +42,10 @@ import java.util.concurrent.locks.ReentrantLock;
  * A manager of object Lifecycles.
  * <p/>
  * This object has methods for registering objects that should be started and stopped.  The Lifecycle allows for
- * two stages: Stage.NORMAL and Stage.LAST.
+ * three stages: Stage.INIT, Stage.NORMAL, and Stage.LAST.
  * <p/>
- * Things added at Stage.NORMAL will be started first (in the order that they are added to the Lifecycle instance) and
- * then things added at Stage.LAST will be started.
+ * Things added at Stage.INIT will be started first (in the order that they are added to the Lifecycle instance) and
+ * then things added at Stage.NORMAL, and finally, Stage.LAST will be started.
  * <p/>
  * The close operation goes in reverse order, starting with the last thing added at Stage.LAST and working backwards.
  * <p/>
@@ -57,6 +59,7 @@ public class Lifecycle
 
   public enum Stage
   {
+    INIT,
     NORMAL,
     LAST
   }
@@ -77,9 +80,17 @@ public class Lifecycle
   private final AtomicReference<State> state = new AtomicReference<>(State.NOT_STARTED);
   private Stage currStage = null;
   private final AtomicBoolean shutdownHookRegistered = new AtomicBoolean(false);
+  private final String name;
 
   public Lifecycle()
   {
+    this("anonymous");
+  }
+
+  public Lifecycle(String name)
+  {
+    Preconditions.checkArgument(StringUtils.isNotEmpty(name), "Lifecycle name must not be null or empty");
+    this.name = name;
     handlers = new TreeMap<>();
     for (Stage stage : Stage.values()) {
       handlers.put(stage, new CopyOnWriteArrayList<>());
@@ -307,10 +318,12 @@ public class Lifecycle
       }
       for (Map.Entry<Stage, ? extends List<Handler>> e : handlers.entrySet()) {
         currStage = e.getKey();
+        log.info("Starting lifecycle [%s] stage [%s]", name, currStage.name());
         for (Handler handler : e.getValue()) {
           handler.start();
         }
       }
+      log.info("Successfully started lifecycle [%s]", name);
     }
     finally {
       startStopLock.unlock();
@@ -323,19 +336,21 @@ public class Lifecycle
     // a simple variable. State change before startStopLock.lock() is needed for the new state visibility during the
     // check in addMaybeStartHandler() marked by (*).
     if (!state.compareAndSet(State.RUNNING, State.STOP)) {
-      log.info("Already stopped and stop was called. Silently skipping");
+      log.info("Lifecycle [%s] already stopped and stop was called. Silently skipping", name);
       return;
     }
     startStopLock.lock();
     try {
       RuntimeException thrown = null;
-      for (List<Handler> stageHandlers : handlers.descendingMap().values()) {
-        for (Handler handler : Lists.reverse(stageHandlers)) {
+
+      for (Stage s : handlers.navigableKeySet().descendingSet()) {
+        log.info("Stopping lifecycle [%s] stage [%s]", name, s.name());
+        for (Handler handler : Lists.reverse(handlers.get(s))) {
           try {
             handler.stop();
           }
           catch (RuntimeException e) {
-            log.warn(e, "exception thrown when stopping %s", handler);
+            log.warn(e, "Lifecycle [%s] encountered exception while stopping %s", name, handler);
             if (thrown == null) {
               thrown = e;
             }
@@ -362,7 +377,7 @@ public class Lifecycle
                 @Override
                 public void run()
                 {
-                  log.info("Running shutdown hook");
+                  log.info("Lifecycle [%s] running shutdown hook", name);
                   stop();
                 }
               }
diff --git a/core/src/test/java/org/apache/druid/java/util/common/lifecycle/LifecycleTest.java b/core/src/test/java/org/apache/druid/java/util/common/lifecycle/LifecycleTest.java
index b1be51b..6354cdd 100644
--- a/core/src/test/java/org/apache/druid/java/util/common/lifecycle/LifecycleTest.java
+++ b/core/src/test/java/org/apache/druid/java/util/common/lifecycle/LifecycleTest.java
@@ -177,19 +177,20 @@ public class LifecycleTest
     lifecycle.addManagedInstance(new ObjectToBeLifecycled(5, startOrder, stopOrder));
     lifecycle.addStartCloseInstance(new ObjectToBeLifecycled(6, startOrder, stopOrder), Lifecycle.Stage.LAST);
     lifecycle.addManagedInstance(new ObjectToBeLifecycled(7, startOrder, stopOrder));
+    lifecycle.addStartCloseInstance(new ObjectToBeLifecycled(8, startOrder, stopOrder), Lifecycle.Stage.INIT);
 
-    final List<Integer> expectedOrder = Arrays.asList(0, 1, 2, 4, 5, 7, 3, 6);
+    final List<Integer> expectedOrder = Arrays.asList(8, 0, 1, 2, 4, 5, 7, 3, 6);
 
     lifecycle.start();
 
-    Assert.assertEquals(8, startOrder.size());
+    Assert.assertEquals(9, startOrder.size());
     Assert.assertEquals(0, stopOrder.size());
     Assert.assertEquals(expectedOrder, startOrder);
 
     lifecycle.stop();
 
-    Assert.assertEquals(8, startOrder.size());
-    Assert.assertEquals(8, stopOrder.size());
+    Assert.assertEquals(9, startOrder.size());
+    Assert.assertEquals(9, stopOrder.size());
     Assert.assertEquals(Lists.reverse(expectedOrder), stopOrder);
   }
 
@@ -212,7 +213,7 @@ public class LifecycleTest
                 new ObjectToBeLifecycled(1, startOrder, stopOrder), Lifecycle.Stage.NORMAL
             );
             lifecycle.addMaybeStartManagedInstance(
-                new ObjectToBeLifecycled(2, startOrder, stopOrder), Lifecycle.Stage.NORMAL
+                new ObjectToBeLifecycled(2, startOrder, stopOrder), Lifecycle.Stage.INIT
             );
             lifecycle.addMaybeStartManagedInstance(
                 new ObjectToBeLifecycled(3, startOrder, stopOrder), Lifecycle.Stage.LAST
@@ -234,6 +235,7 @@ public class LifecycleTest
     );
 
     final List<Integer> expectedOrder = Arrays.asList(0, 1, 2, 4, 5, 7, 3, 6);
+    final List<Integer> expectedStopOrder = Arrays.asList(6, 3, 7, 5, 4, 1, 0, 2);
 
     lifecycle.start();
 
@@ -243,7 +245,7 @@ public class LifecycleTest
     lifecycle.stop();
 
     Assert.assertEquals(expectedOrder, startOrder);
-    Assert.assertEquals(Lists.reverse(expectedOrder), stopOrder);
+    Assert.assertEquals(expectedStopOrder, stopOrder);
   }
 
   public static class ObjectToBeLifecycled
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java
index 25fe06f..fd53c76 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskMaster.java
@@ -120,7 +120,7 @@ public class TaskMaster implements TaskCountStatsProvider
           );
 
           // Sensible order to start stuff:
-          final Lifecycle leaderLifecycle = new Lifecycle();
+          final Lifecycle leaderLifecycle = new Lifecycle("task-master");
           if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) {
             log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared!  Race condition")
                .emit();
@@ -166,6 +166,7 @@ public class TaskMaster implements TaskCountStatsProvider
         try {
           initialized = false;
           final Lifecycle leaderLifecycle = leaderLifecycleRef.getAndSet(null);
+
           if (leaderLifecycle != null) {
             leaderLifecycle.stop();
           }
@@ -203,6 +204,7 @@ public class TaskMaster implements TaskCountStatsProvider
     giant.lock();
 
     try {
+      gracefulStopLeaderLifecycle();
       overlordLeaderSelector.unregisterListener();
     }
     finally {
@@ -322,4 +324,16 @@ public class TaskMaster implements TaskCountStatsProvider
       return null;
     }
   }
+
+  private void gracefulStopLeaderLifecycle()
+  {
+    try {
+      if (isLeader()) {
+        leadershipListener.stopBeingLeader();
+      }
+    }
+    catch (Exception ex) {
+      // fail silently since we are stopping anyway
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java b/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java
index 4140eaf..c5c3428 100644
--- a/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java
+++ b/server/src/main/java/org/apache/druid/guice/http/LifecycleUtils.java
@@ -26,7 +26,7 @@ public class LifecycleUtils
 {
   public static Lifecycle asMmxLifecycle(Lifecycle lifecycle)
   {
-    final Lifecycle metamxLifecycle = new Lifecycle();
+    final Lifecycle metamxLifecycle = new Lifecycle("http-client");
     try {
       lifecycle.addMaybeStartHandler(new Lifecycle.Handler()
       {
diff --git a/server/src/main/java/org/apache/druid/initialization/Log4jShutterDownerModule.java b/server/src/main/java/org/apache/druid/initialization/Log4jShutterDownerModule.java
index ee13f9b..103f03f 100644
--- a/server/src/main/java/org/apache/druid/initialization/Log4jShutterDownerModule.java
+++ b/server/src/main/java/org/apache/druid/initialization/Log4jShutterDownerModule.java
@@ -25,7 +25,7 @@ import com.google.inject.Module;
 import com.google.inject.Provides;
 import com.google.inject.name.Names;
 import org.apache.druid.common.config.Log4jShutdown;
-import org.apache.druid.guice.ManageLifecycle;
+import org.apache.druid.guice.ManageLifecycleInit;
 import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
 import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.common.logger.Logger;
@@ -41,8 +41,8 @@ public class Log4jShutterDownerModule implements Module
   @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.
+    // Instantiate eagerly so that we get everything registered and put into the Lifecycle as early as possible
+    // Lifecycle scope is INIT to ensure stop runs in the last phase of lifecycle stop.
 
     try {
       ClassLoader loader = Thread.currentThread().getContextClassLoader();
@@ -84,7 +84,7 @@ public class Log4jShutterDownerModule implements Module
   }
 
 
-  @ManageLifecycle
+  @ManageLifecycleInit
   @Provides
   public Log4jShutterDowner getShutterDowner(
       Log4jShutdown log4jShutdown


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org