This is an automated email from the ASF dual-hosted git repository. abhishekrb pushed a commit to branch more_lifecycle_logging in repository https://gitbox.apache.org/repos/asf/druid.git
commit 2101672a2c4d1214b28986157c73b44b10f1a1d2 Author: Abhishek Balaji Radhakrishnan <[email protected]> AuthorDate: Fri Oct 17 17:35:21 2025 -0700 Log all the handlers for a stage before they start / stop. This logs all the handlers at the start/stop lifecycle stage. This is useful to know the order ahead of time, especially when a service stops, some handlers don't execute. --- .../java/util/common/lifecycle/Lifecycle.java | 32 ++++++++++++++++++---- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/processing/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java index a91038c00eb..c4c9691189b 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java @@ -337,8 +337,9 @@ 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()) { + final List<Handler> handlersToStart = e.getValue(); + log.info("Starting lifecycle [%s] stage [%s] for handlers [%s]", name, currStage.name(), handlersToStart); + for (Handler handler : handlersToStart) { handler.start(); } } @@ -363,13 +364,14 @@ public class Lifecycle Exception thrown = null; for (Stage s : handlers.navigableKeySet().descendingSet()) { - log.info("Stopping lifecycle [%s] stage [%s]", name, s.name()); - for (Handler handler : Lists.reverse(handlers.get(s))) { + final List<Handler> handlersToStop = Lists.reverse(handlers.get(s)); + log.info("Stopping lifecycle [%s] stage [%s] for handlers [%s]", name, s.name(), handlersToStop); + for (Handler handler : handlersToStop) { try { handler.stop(); } catch (Exception e) { - log.warn(e, "Lifecycle [%s] encountered exception while stopping %s", name, handler); + log.warn(e, "Lifecycle [%s] encountered exception while stopping [%s]", name, handler); if (thrown == null) { thrown = e; } else { @@ -476,6 +478,12 @@ public class Lifecycle } } } + + @Override + public String toString() + { + return o.getClass().getSimpleName(); + } } private static class StartCloseHandler implements Handler @@ -514,9 +522,15 @@ public class Lifecycle stopMethod.invoke(o); } catch (Exception e) { - log.error(e, "Unable to invoke stopMethod() on %s", o.getClass()); + log.error(e, "Unable to invoke stopMethod() on [%s]", o.getClass()); } } + + @Override + public String toString() + { + return o.getClass().getSimpleName(); + } } private static class CloseableHandler implements Handler @@ -546,5 +560,11 @@ public class Lifecycle log.error(e, "Exception when closing object [%s]", o); } } + + @Override + public String toString() + { + return o.getClass().getSimpleName(); + } } } --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
