You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2019/05/21 15:46:40 UTC

[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #4222: Add startup/shutdown checks for HelixServerStarter

mcvsubbu commented on a change in pull request #4222: Add startup/shutdown checks for HelixServerStarter
URL: https://github.com/apache/incubator-pinot/pull/4222#discussion_r286097130
 
 

 ##########
 File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
 ##########
 @@ -352,135 +287,160 @@ public void stop() {
     _adminApiApplication.stop();
     setShuttingDownStatus(true);
 
-    // Total waiting time should include max query time.
-    final long endTime = _maxShutdownWaitTimeMs + System.currentTimeMillis();
-    if (_helixServerConfig.getBoolean(CommonConstants.Server.CONFIG_OF_ENABLE_SHUTDOWN_DELAY, true)) {
-      Uninterruptibles.sleepUninterruptibly(_maxQueryTimeMs, TimeUnit.MILLISECONDS);
+    long endTimeMs = startTimeMs + _serverConf.getLong(CONFIG_OF_SHUTDOWN_TIMEOUT_MS, DEFAULT_SHUTDOWN_TIMEOUT_MS);
+    if (_serverConf.getBoolean(CONFIG_OF_SHUTDOWN_ENABLE_QUERY_CHECK, DEFAULT_SHUTDOWN_ENABLE_QUERY_CHECK)) {
+      shutdownQueryCheck(endTimeMs);
     }
-    waitUntilNoIncomingQueries(System.currentTimeMillis(), endTime);
     _helixManager.disconnect();
     _serverInstance.shutDown();
-    waitUntilNoOnlineResources(System.currentTimeMillis(), endTime);
-  }
-
-  private void waitUntilNoIncomingQueries(long startTime, final long endTime) {
-    if (startTime >= endTime) {
-      LOGGER.warn("Skip waiting until no incoming queries.");
-      return;
+    if (_serverConf.getBoolean(CONFIG_OF_SHUTDOWN_ENABLE_RESOURCE_CHECK, DEFAULT_SHUTDOWN_ENABLE_RESOURCE_CHECK)) {
+      shutdownResourceCheck(endTimeMs);
     }
-    LOGGER.info("Waiting upto {}ms until Pinot server doesn't receive any incoming queries...", (endTime - startTime));
-    long currentTime = startTime;
+  }
 
-    while (currentTime < endTime) {
-      if (noIncomingQueries(currentTime)) {
-        LOGGER.info("No incoming query within {}ms. Total waiting Time: {}ms", _checkIntervalTimeMs,
-            (currentTime - startTime));
-        return;
+  /**
+   * When shutting down the server, drains the queries and waits for all the existing queries to be finished.
+   *
+   * @param endTimeMs Timeout for the check
+   */
+  private void shutdownQueryCheck(long endTimeMs) {
+    LOGGER.info("Starting shutdown query check");
+    long startTimeMs = System.currentTimeMillis();
+
+    long maxQueryTimeMs = _serverConf.getLong(CONFIG_OF_QUERY_EXECUTOR_TIMEOUT, DEFAULT_QUERY_EXECUTOR_TIMEOUT_MS);
+    long noQueryThresholdMs = _serverConf.getLong(CONFIG_OF_SHUTDOWN_NO_QUERY_THRESHOLD_MS, maxQueryTimeMs);
+
+    // Drain queries
+    boolean queriesDrained = false;
+    long currentTimeMs;
+    while ((currentTimeMs = System.currentTimeMillis()) < endTimeMs) {
+      long latestQueryTimeMs = _serverInstance.getLatestQueryTime();
+      if (currentTimeMs >= latestQueryTimeMs + noQueryThresholdMs) {
+        LOGGER.info("Finished draining queries (no query received within {}ms) after {}ms",
+            currentTimeMs - latestQueryTimeMs, currentTimeMs - startTimeMs);
+        queriesDrained = true;
+        break;
       }
-
       try {
-        Thread.sleep(Math.min(_maxQueryTimeMs, (endTime - currentTime)));
+        Thread.sleep(Math.min(noQueryThresholdMs - latestQueryTimeMs, endTimeMs - currentTimeMs));
 
 Review comment:
   noQueryThresholdMs is an interval time, and latestQueryTime is an absolute value. Am I reading it right that we are subtracting one from the other and using that as an interval?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org