gianm commented on code in PR #15726:
URL: https://github.com/apache/druid/pull/15726#discussion_r1504695530
##########
server/src/main/java/org/apache/druid/discovery/BaseNodeRoleWatcher.java:
##########
@@ -71,22 +71,35 @@ public class BaseNodeRoleWatcher
private volatile boolean cacheInitializationTimedOut = false;
public BaseNodeRoleWatcher(
- ExecutorService listenerExecutor,
+ ScheduledExecutorService listenerExecutor,
NodeRole nodeRole
)
{
- this.listenerExecutor = listenerExecutor;
this.nodeRole = nodeRole;
+ this.listenerExecutor = listenerExecutor;
+ this.listenerExecutor.schedule(
+ () -> {
+ checkCacheInitialization(1L);
+ },
+ 30,
+ TimeUnit.SECONDS
+ );
}
public Collection<DiscoveryDruidNode> getAllNodes()
{
if (cacheInitializationTimedOut) {
return unmodifiableNodes;
}
+ checkCacheInitialization(30L);
+ return unmodifiableNodes;
+ }
+
+ private void checkCacheInitialization(long timeoutSeconds)
+ {
boolean nodeViewInitialized;
try {
- nodeViewInitialized = cacheInitialized.await((long) 30,
TimeUnit.SECONDS);
+ nodeViewInitialized = cacheInitialized.await(timeoutSeconds,
TimeUnit.SECONDS);
Review Comment:
We can't call `await` while holding the `listenerExecutor`, because it needs
to remain free for other runnables to run. So it's important not to block the
executor thread. The pattern should be:
- In the constructor, schedule something for 30s out, that triggers
initialization-timed-out if the cache hasn't yet initialized. It should set the
volatile and count down the latch.
- In `getAllNodes()`, do `await` same as it used to do.
--
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]