kfaraz commented on code in PR #14517:
URL: https://github.com/apache/druid/pull/14517#discussion_r1252613089


##########
server/src/main/java/org/apache/druid/server/coordination/ChangeRequestHttpSyncer.java:
##########
@@ -144,69 +150,91 @@ public void stop()
   {
     synchronized (startStopLock) {
       if (!startStopLock.canStop()) {
-        throw new ISE("Can't stop ChangeRequestHttpSyncer[%s].", logIdentity);
+        throw new ISE("Could not stop sync for server[%s].", logIdentity);
       }
       try {
-        log.info("Stopping ChangeRequestHttpSyncer[%s].", logIdentity);
+        log.info("Stopping sync for server[%s].", logIdentity);
       }
       finally {
         startStopLock.exitStop();
       }
 
-      log.info("Stopped ChangeRequestHttpSyncer[%s].", logIdentity);
+      log.info("Stopped sync for server[%s].", logIdentity);
     }
   }
 
-  /** Wait for first fetch of segment listing from server. */
-  public boolean awaitInitialization(long timeout, TimeUnit timeUnit) throws 
InterruptedException
+  /**
+   * Waits for the first successful sync with this server up to {@link 
#maxDurationToWaitForSync}.
+   */
+  public boolean awaitInitialization() throws InterruptedException
   {
-    return initializationLatch.await(timeout, timeUnit);
+    return initializationLatch.await(maxDurationToWaitForSync.getMillis(), 
TimeUnit.MILLISECONDS);
   }
 
   /**
-   * This method returns the debugging information for printing, must not be 
used for any other purpose.
+   * Waits upto 1 millisecond for the first successful sync with this server.
    */
-  public Map<String, Object> getDebugInfo()
+  public boolean isInitialized() throws InterruptedException
   {
-    long currTime = System.currentTimeMillis();
+    return initializationLatch.await(1, TimeUnit.MILLISECONDS);
+  }
 
-    Object notSuccessfullySyncedFor;
-    if (lastSuccessfulSyncTime == 0) {
-      notSuccessfullySyncedFor = "Never Successfully Synced";
-    } else {
-      notSuccessfullySyncedFor = (currTime - lastSuccessfulSyncTime) / 1000;
-    }
+  /**
+   * Returns debugging information for printing, must not be used for any 
other purpose.
+   */
+  public Map<String, Object> getDebugInfo()
+  {
     return ImmutableMap.of(
-        "notSyncedForSecs", lastSyncTime == 0 ? "Never Synced" : (currTime - 
lastSyncTime) / 1000,
-        "notSuccessfullySyncedFor", notSuccessfullySyncedFor,
+        "millisSinceLastRequest", sinceLastSyncRequest.millisElapsed(),
+        "millisSinceLastSuccess", sinceLastSyncSuccess.millisElapsed(),
         "consecutiveFailedAttemptCount", consecutiveFailedAttemptCount,
         "syncScheduled", startStopLock.isStarted()
     );
   }
 
   /**
-   * Exposed for monitoring use to see if sync is working fine and not stopped 
due to any coding bugs. If this
-   * ever returns false then caller of this method must create an alert and it 
should be looked into for any
-   * bugs.
+   * Whether this syncer should be reset. This method returning true typically
+   * indicates a problem with the sync scheduler.
+   *
+   * @return true if the delay since the last request to the server (or since
+   * syncer start in case of no request to the server) has exceeded
+   * {@link #maxDelayBetweenSyncRequests}.
    */
-  public boolean isOK()
+  public boolean needsReset()
   {
-    return (System.currentTimeMillis() - lastSyncTime) < MAX_RETRY_BACKOFF + 3 
* serverHttpTimeout;
+    if (sinceLastSyncRequest.isRunning()) {
+      return sinceLastSyncRequest.hasElapsed(maxDelayBetweenSyncRequests);
+    } else {
+      return sinceSyncerStart.hasElapsed(maxDelayBetweenSyncRequests);
+    }
   }
 
-  public long getServerHttpTimeout()
+  public long getUnstableTimeMillis()
   {
-    return serverHttpTimeout;
+    return consecutiveFailedAttemptCount <= 0 ? 0 : 
sinceUnstable.millisElapsed();
+  }
+
+  /**
+   * @return true if there have been no sync failures recently and the last
+   * successful sync was not more than {@link #maxDurationToWaitForSync} ago.
+   */
+  public boolean isSyncedSuccessfully()
+  {
+    if (consecutiveFailedAttemptCount > 0) {
+      return false;
+    } else {
+      return sinceLastSyncSuccess.hasNotElapsed(maxDurationToWaitForSync);
+    }
   }
 
   private void sync()
   {
     if (!startStopLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
-      log.info("Skipping sync() call for server[%s].", logIdentity);
+      log.info("Skipping sync for server[%s] as lifecycle has not started.", 
logIdentity);

Review Comment:
   I am updating the message to say "as syncer has not started yet" as the 
`startStopLock` denotes nothing but the lifecycle of the syncer itself. 



-- 
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