abhishekagarwal87 commented on code in PR #15726:
URL: https://github.com/apache/druid/pull/15726#discussion_r1512311579


##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -215,34 +229,73 @@ public void cacheInitialized()
       // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
       // counted down.
       if (cacheInitialized.getCount() == 0) {
-        LOGGER.error("cache is already initialized. ignoring cache 
initialization event.");
+        if (cacheInitializationTimedOut) {
+          LOGGER.warn(
+              "Cache initialization for node role[%s] has already timed out. 
Ignoring cacheInitialized event.",

Review Comment:
   ```suggestion
                 "Cache initialization for node role[%s] has already timed out. 
Ignoring cache initialization event.",
   ```



##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -215,34 +229,73 @@ public void cacheInitialized()
       // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
       // counted down.
       if (cacheInitialized.getCount() == 0) {
-        LOGGER.error("cache is already initialized. ignoring cache 
initialization event.");
+        if (cacheInitializationTimedOut) {
+          LOGGER.warn(
+              "Cache initialization for node role[%s] has already timed out. 
Ignoring cacheInitialized event.",
+              nodeRole.getJsonName()
+          );
+        } else {
+          LOGGER.error(
+              "Cache for node role[%s] is already initialized. ignoring cache 
initialization event.",
+              nodeRole.getJsonName()
+          );
+        }
         return;
       }
 
-      // It is important to take a snapshot here as list of nodes might change 
by the time listeners process
-      // the changes.
-      List<DiscoveryDruidNode> currNodes = Lists.newArrayList(nodes.values());
-      LOGGER.info(
-          "Node watcher of role [%s] is now initialized with %d nodes.",
-          nodeRole.getJsonName(),
-          currNodes.size());
+      cacheInitialized(false);
+    }
+  }
 
-      for (DruidNodeDiscovery.Listener listener : nodeListeners) {
-        safeSchedule(
-            () -> {
-              listener.nodesAdded(currNodes);
-              listener.nodeViewInitialized();
-            },
-            "Exception occurred in nodesAdded([%s]) in listener [%s].",
-            currNodes,
-            listener
-        );
+  private void cacheInitializedTimedOut()
+  {
+    synchronized (lock) {
+      // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
+      // counted down.
+      if (cacheInitialized.getCount() == 0) {
+        LOGGER.warn("Cache for node watcher of role[%s] is already 
initialized. ignoring timeout.", nodeRole.getJsonName());
+        return;
       }
 
-      cacheInitialized.countDown();
+      cacheInitialized(true);
     }
   }
 
+  @GuardedBy("lock")
+  private void cacheInitialized(boolean timedOut)
+  {
+    if (timedOut) {
+      LOGGER.warn("Cache for node role [%s] could not be initialized before 
timeout.", nodeRole.getJsonName());
+      cacheInitializationTimedOut = true;
+    }
+

Review Comment:
   Please also add some warning if the list of nodes is empty for whatever 
reason. 



##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -215,34 +229,73 @@ public void cacheInitialized()
       // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
       // counted down.
       if (cacheInitialized.getCount() == 0) {
-        LOGGER.error("cache is already initialized. ignoring cache 
initialization event.");
+        if (cacheInitializationTimedOut) {
+          LOGGER.warn(
+              "Cache initialization for node role[%s] has already timed out. 
Ignoring cacheInitialized event.",
+              nodeRole.getJsonName()
+          );
+        } else {
+          LOGGER.error(
+              "Cache for node role[%s] is already initialized. ignoring cache 
initialization event.",
+              nodeRole.getJsonName()
+          );
+        }
         return;
       }
 
-      // It is important to take a snapshot here as list of nodes might change 
by the time listeners process
-      // the changes.
-      List<DiscoveryDruidNode> currNodes = Lists.newArrayList(nodes.values());
-      LOGGER.info(
-          "Node watcher of role [%s] is now initialized with %d nodes.",
-          nodeRole.getJsonName(),
-          currNodes.size());
+      cacheInitialized(false);
+    }
+  }
 
-      for (DruidNodeDiscovery.Listener listener : nodeListeners) {
-        safeSchedule(
-            () -> {
-              listener.nodesAdded(currNodes);
-              listener.nodeViewInitialized();
-            },
-            "Exception occurred in nodesAdded([%s]) in listener [%s].",
-            currNodes,
-            listener
-        );
+  private void cacheInitializedTimedOut()
+  {
+    synchronized (lock) {
+      // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
+      // counted down.
+      if (cacheInitialized.getCount() == 0) {
+        LOGGER.warn("Cache for node watcher of role[%s] is already 
initialized. ignoring timeout.", nodeRole.getJsonName());
+        return;
       }
 
-      cacheInitialized.countDown();
+      cacheInitialized(true);
     }
   }
 
+  @GuardedBy("lock")
+  private void cacheInitialized(boolean timedOut)

Review Comment:
   can you add a comment here that this method is only called once? 



##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -215,34 +229,73 @@ public void cacheInitialized()
       // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
       // counted down.
       if (cacheInitialized.getCount() == 0) {
-        LOGGER.error("cache is already initialized. ignoring cache 
initialization event.");
+        if (cacheInitializationTimedOut) {
+          LOGGER.warn(
+              "Cache initialization for node role[%s] has already timed out. 
Ignoring cacheInitialized event.",
+              nodeRole.getJsonName()
+          );
+        } else {
+          LOGGER.error(
+              "Cache for node role[%s] is already initialized. ignoring cache 
initialization event.",
+              nodeRole.getJsonName()
+          );
+        }
         return;
       }
 
-      // It is important to take a snapshot here as list of nodes might change 
by the time listeners process
-      // the changes.
-      List<DiscoveryDruidNode> currNodes = Lists.newArrayList(nodes.values());
-      LOGGER.info(
-          "Node watcher of role [%s] is now initialized with %d nodes.",
-          nodeRole.getJsonName(),
-          currNodes.size());
+      cacheInitialized(false);
+    }
+  }
 
-      for (DruidNodeDiscovery.Listener listener : nodeListeners) {
-        safeSchedule(
-            () -> {
-              listener.nodesAdded(currNodes);
-              listener.nodeViewInitialized();
-            },
-            "Exception occurred in nodesAdded([%s]) in listener [%s].",
-            currNodes,
-            listener
-        );
+  private void cacheInitializedTimedOut()
+  {
+    synchronized (lock) {
+      // No need to wait on CountDownLatch, because we are holding the lock 
under which it could only be
+      // counted down.
+      if (cacheInitialized.getCount() == 0) {
+        LOGGER.warn("Cache for node watcher of role[%s] is already 
initialized. ignoring timeout.", nodeRole.getJsonName());
+        return;
       }
 
-      cacheInitialized.countDown();
+      cacheInitialized(true);
     }
   }
 
+  @GuardedBy("lock")
+  private void cacheInitialized(boolean timedOut)
+  {
+    if (timedOut) {
+      LOGGER.warn("Cache for node role [%s] could not be initialized before 
timeout.", nodeRole.getJsonName());

Review Comment:
   ```suggestion
         LOGGER.warn("Cache for node role [%s] could not be initialized before 
timeout. This service may not have full information about other nodes of type 
[%s].", nodeRole.getJsonName());
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to