You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tez.apache.org by GitBox <gi...@apache.org> on 2021/01/31 05:50:27 UTC

[GitHub] [tez] rbalamohan commented on a change in pull request #97: TEZ-4277: AsyncDispatcher can hang on serviceStop if the eventhandling thread is in BLOCKED state

rbalamohan commented on a change in pull request #97:
URL: https://github.com/apache/tez/pull/97#discussion_r567370485



##########
File path: tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java
##########
@@ -142,19 +143,34 @@ protected void serviceStop() throws Exception {
     if (drainEventsOnStop) {
       blockNewEvents = true;
       LOG.info("AsyncDispatcher is draining to stop, ignoring any new events.");
+      long endTime = System.currentTimeMillis() + getConfig()
+          .getInt(TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT,
+              TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT);
+
       synchronized (waitForDrained) {
-        while (!drained && eventHandlingThread.isAlive()) {
+        while (!drained && eventHandlingThread.isAlive() && System.currentTimeMillis() < endTime) {
           waitForDrained.wait(1000);
-          LOG.info("Waiting for AsyncDispatcher to drain.");
+          LOG.info(
+              "Waiting for AsyncDispatcher to drain. Current queue size: {}, handler thread state: {}",
+              eventQueue.size(), eventHandlingThread.getState());
         }
       }
-      
     }
     stopped = true;
     if (eventHandlingThread != null) {
       eventHandlingThread.interrupt();

Review comment:
       Thanks for sharing the patch. Overall looks good. Minor comments below.
   
   If you look at eventHandlingThread, it would be making a call to "protected void dispatch(Event event)". This method catches "throwable" and does not reset Thread's interrupt status. So in case, interrupt happened during dispatch, it would be silently gobbled up.
   
   Can you add snippet to catch "InterruptedException" in asyncDispatcher::dispatch(), log that it got interrupted and just reset thread's interrupt status with ("Thread.currentThread().interrupt();"). This would ensure that the next iteration in event handler thread gets a chance to bail out.




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