gianm commented on code in PR #15726:
URL: https://github.com/apache/druid/pull/15726#discussion_r1514832437
##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -215,34 +232,79 @@ 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 cache initialization 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());
Review Comment:
This warning should be removed. In the current code, when the cache
initializes normally this will be logged after 30s. That'll be confusing for
people, since people don't expect warnings to be logged when nothing bad is
happening.
##########
server/src/test/java/org/apache/druid/discovery/BaseNodeRoleWatcherTest.java:
##########
@@ -116,6 +136,69 @@ public void testGeneralUseSimulation()
assertListener(listener3, true, nodesAdded, nodesRemoved);
}
+ @Test(timeout = 60_000L)
+ public void testRegisterListenerBeforeTimeout() throws InterruptedException
+ {
+ BaseNodeRoleWatcher nodeRoleWatcher = new BaseNodeRoleWatcher(exec,
NodeRole.BROKER, 1);
Review Comment:
There's a timing race in this test: we have a timeout of 1 second but
nothing really guarantees that `registerListener` is called prior to the
timeout firing. `testGetAllNodesBeforeTimeout` has a similar problem. It is
also not ideal that there is a `sleep(1500)`, for similar reasons. On busy CI
servers there can be random pauses that cause these tests to fail, test the
wrong thing, or otherwise not work properly.
In both cases, we can fix it by splitting up the timeout-scheduler from the
constructor. This would work:
- Move the `listenerExecutor.schedule` call for
`this::cacheInitializedTimedOut` out of the constructor, put it in some new
package-private method like `void scheduleTimeout(long timeout)`.
- Make the constructor itself package-private.
- Add a static factory method like `create(ScheduledExecutorService,
NodeRole)` that constructs an instance and then calls
`scheduleTimeout(DEFAULT_TIMEOUT)` on it. Use this one in production code.
- Add a package-private `awaitInitialization()` method that awaits the
initialization latch.
- In test code, call the regular constructor (which doesn't schedule the
timeout), then do the pre-timeout part of the test, then call
`scheduleTimeout(0)` (so we time out immediately), then call
`awaitInitialization()`, then do the post-initialization part of the test.
With this structure, the test should complete in a few milliseconds and be
robust to any potential runtime timing issues.
--
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]