You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ab...@apache.org on 2022/02/22 09:33:11 UTC

[tez] branch branch-0.9 updated: TEZ-4388: Asynchttpclient can cause stuck TezChild processes (#189) (Laszlo Bodor reviewed by Rajesh Balamohan)

This is an automated email from the ASF dual-hosted git repository.

abstractdog pushed a commit to branch branch-0.9
in repository https://gitbox.apache.org/repos/asf/tez.git


The following commit(s) were added to refs/heads/branch-0.9 by this push:
     new 75f753e  TEZ-4388: Asynchttpclient can cause stuck TezChild processes (#189) (Laszlo Bodor reviewed by Rajesh Balamohan)
75f753e is described below

commit 75f753e22ab489d0fc1ed9dd8f1f42dd97772e2a
Author: Bodor Laszlo <bo...@gmail.com>
AuthorDate: Tue Feb 22 10:32:30 2022 +0100

    TEZ-4388: Asynchttpclient can cause stuck TezChild processes (#189) (Laszlo Bodor reviewed by Rajesh Balamohan)
---
 .../apache/tez/util/TezRuntimeShutdownHandler.java | 44 ++++++++++++++++++++++
 .../java/org/apache/tez/runtime/task/TezChild.java | 10 ++++-
 .../tez/http/async/netty/AsyncHttpConnection.java  | 11 ++++++
 3 files changed, 63 insertions(+), 2 deletions(-)

diff --git a/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java
new file mode 100644
index 0000000..4881e08
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class TezRuntimeShutdownHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(TezRuntimeShutdownHandler.class);
+
+  private static final List<Runnable> shutdownTasks = new ArrayList<>();
+
+  private TezRuntimeShutdownHandler() {
+  }
+
+  public static void addShutdownTask(Runnable r) {
+    shutdownTasks.add(r);
+  }
+
+  public static synchronized void shutdown() {
+    LOG.info("Handling {} shutdown tasks", shutdownTasks.size());
+    for (Runnable shutdownTask : shutdownTasks) {
+      shutdownTask.run();
+    }
+  }
+}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index f11a034..6b87883 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -72,6 +72,7 @@ import org.apache.tez.runtime.api.ExecutionContext;
 import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
 import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
 import org.apache.tez.runtime.internals.api.TaskReporterInterface;
+import org.apache.tez.util.TezRuntimeShutdownHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -389,8 +390,10 @@ public class TezChild {
       LOG.info("Shutting down container {}", containerIdString);
       // It's possible that there's pending tasks on the executor. Those should be cancelled.
       List<Runnable> pendingRunnables = executor.shutdownNow();
+      LOG.info("There are {} runnables in shared executor, cancelling those...", pendingRunnables.size());
       for (Runnable r : pendingRunnables) {
-        LOG.info("Cancelling pending runnables during TezChild shutdown for containerId={}", containerIdString);
+        LOG.info("Cancelling pending runnable ({}) during TezChild shutdown for containerId={}", r.hashCode(),
+            containerIdString);
         ((FutureTask)r).cancel(false);
       }
       if (taskReporter != null) {
@@ -400,6 +403,8 @@ public class TezChild {
         RPC.stopProxy(umbilical);
       }
     }
+    TezRuntimeShutdownHandler.shutdown();
+    LOG.info("TezChild shutdown finished");
   }
 
   public static class ContainerExecutionResult {
@@ -521,7 +526,8 @@ public class TezChild {
         System.getenv(), pid, new ExecutionContextImpl(System.getenv(Environment.NM_HOST.name())),
         credentials, Runtime.getRuntime().maxMemory(), System
             .getenv(ApplicationConstants.Environment.USER.toString()), null, true, hadoopShim);
-    tezChild.run();
+    ContainerExecutionResult result = tezChild.run();
+    LOG.info("TezChild is about to exit from main(), run() returned result: {}", result.toString());
   }
 
   private void handleError(Throwable t) {
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
index feae8ab..46cb8b2 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
@@ -36,6 +36,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
 import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
 import org.apache.tez.util.StopWatch;
+import org.apache.tez.util.TezRuntimeShutdownHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -103,6 +104,16 @@ public class AsyncHttpConnection extends BaseHttpConnection {
               .build();
           DefaultAsyncHttpClientConfig config = builder.build();
           httpAsyncClient = new DefaultAsyncHttpClient(config);
+          TezRuntimeShutdownHandler.addShutdownTask(() -> {
+            try {
+              if (httpAsyncClient != null) {
+                httpAsyncClient.close();
+                httpAsyncClient = null;
+              }
+            } catch (IOException e) {
+              LOG.warn("Error while closing async client (this won't block shutdown)", e);
+            }
+          });
         }
       }
     }