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 2023/04/20 11:13:31 UTC

[tez] branch master updated: TEZ-4488: TaskSchedulerManager might not be initialized when the first DAG comes (#280) (Laszlo Bodor reviewed by Rajesh Balamohan)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 249e017b3 TEZ-4488: TaskSchedulerManager might not be initialized when the first DAG comes (#280) (Laszlo Bodor reviewed by Rajesh Balamohan)
249e017b3 is described below

commit 249e017b3f103874f819d2aec42c1fa88e2fa7a4
Author: Bodor Laszlo <bo...@gmail.com>
AuthorDate: Thu Apr 20 13:13:25 2023 +0200

    TEZ-4488: TaskSchedulerManager might not be initialized when the first DAG comes (#280) (Laszlo Bodor reviewed by Rajesh Balamohan)
---
 .../org/apache/tez/dag/api/TezConfiguration.java   | 11 +++
 .../java/org/apache/tez/dag/app/DAGAppMaster.java  | 14 ++++
 .../tez/dag/app/DAGAppMasterReadinessService.java  | 83 ++++++++++++++++++++++
 .../org/apache/tez/dag/app/TestDAGAppMaster.java   |  6 ++
 4 files changed, 114 insertions(+)

diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 6d7783624..30f33e26a 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -909,6 +909,17 @@ public class TezConfiguration extends Configuration {
   public static final String TEZ_AM_YARN_SCHEDULER_CLASS_DEFAULT =
       "org.apache.tez.dag.app.rm.YarnTaskSchedulerService";
 
+  /**
+   * Int value. The AM waits this amount of time when the first DAG is submitted but not all the services are ready.
+   * This can happen when the client RPC handler is up and able to accept DAGs but e.g. task scheduler
+   * manager is not ready (e.g. a task scheduler is waiting for external resources).
+   * A value equal or less than 0 is not supported and leads to an exception.
+   */
+  @ConfigurationScope(Scope.AM)
+  @ConfigurationProperty
+  public static final String TEZ_AM_READY_FOR_SUBMIT_TIMEOUT_MS = TEZ_AM_PREFIX + "ready.for.submit.timeout.ms";
+  public static final int TEZ_AM_READY_FOR_SUBMIT_TIMEOUT_MS_DEFAULT = 30000;
+
   /** Int value. The amount of memory in MB to be used by the AppMaster */
   @ConfigurationScope(Scope.AM)
   @ConfigurationProperty(type="integer")
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 2ef72f8c5..e9304599d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -263,6 +263,7 @@ public class DAGAppMaster extends AbstractService {
   private DagEventDispatcher dagEventDispatcher;
   private VertexEventDispatcher vertexEventDispatcher;
   private TaskSchedulerManager taskSchedulerManager;
+  private DAGAppMasterReadinessService appMasterReadinessService;
   private WebUIService webUIService;
   private HistoryEventHandler historyEventHandler;
   private final Map<String, LocalResource> amResources = new HashMap<String, LocalResource>();
@@ -587,6 +588,8 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerManager);
     addIfServiceDependency(taskSchedulerManager, clientRpcServer);
 
+    appMasterReadinessService = createAppMasterReadinessService();
+
     this.containerLauncherManager = createContainerLauncherManager(containerLauncherDescriptors,
         isLocal);
     addIfService(containerLauncherManager, true);
@@ -662,6 +665,15 @@ public class DAGAppMaster extends AbstractService {
         taskSchedulerDescriptors, isLocal, hadoopShim);
   }
 
+  @VisibleForTesting
+  protected DAGAppMasterReadinessService createAppMasterReadinessService() {
+    DAGAppMasterReadinessService service =
+        new DAGAppMasterReadinessService(DAGAppMasterReadinessService.class.getName());
+    addIfService(service, false);
+    addIfServiceDependency(service, taskSchedulerManager);
+    return service;
+  }
+
   @VisibleForTesting
   protected ContainerSignatureMatcher createContainerSignatureMatcher() {
     return new ContainerContextMatcher();
@@ -1291,6 +1303,8 @@ public class DAGAppMaster extends AbstractService {
 
   public String submitDAGToAppMaster(DAGPlan dagPlan,
       Map<String, LocalResource> additionalResources) throws TezException {
+    appMasterReadinessService.waitToBeReady();
+
     if (sessionStopped.get()) {
       throw new SessionNotRunning("AM unable to accept new DAG submissions."
           + " In the process of shutting down");
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMasterReadinessService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMasterReadinessService.java
new file mode 100644
index 000000000..cd7dff0f6
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMasterReadinessService.java
@@ -0,0 +1,83 @@
+/**
+* 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
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* 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.dag.app;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is an artifical service to be used in DAGAppMaster,
+ * which can be added to have dependencies that are crucial in order to be
+ * able to run DAGs.
+ *
+ */
+public class DAGAppMasterReadinessService extends AbstractService {
+  private static final Logger LOG = LoggerFactory.getLogger(DAGAppMasterReadinessService.class);
+
+  private AtomicBoolean ready = new AtomicBoolean(false);
+  private int timeoutMs;
+
+  public DAGAppMasterReadinessService(String name) {
+    super(name);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    timeoutMs = getConfig().getInt(TezConfiguration.TEZ_AM_READY_FOR_SUBMIT_TIMEOUT_MS,
+        TezConfiguration.TEZ_AM_READY_FOR_SUBMIT_TIMEOUT_MS_DEFAULT);
+    if (timeoutMs <= 0) {
+      throw new TezException(
+          "timeout <= 0 is not supported for " + TezConfiguration.TEZ_AM_READY_FOR_SUBMIT_TIMEOUT_MS);
+    }
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    ready.set(true);
+  }
+
+  /**
+   * The waitToBeReady waits until this service really starts. When the serviceStart
+   * is called and this service is ready, we can make sure that the dependency services
+   * has already been started too.
+   * @throws TezException
+   */
+  public void waitToBeReady() throws TezException {
+    long start = System.currentTimeMillis();
+    while (!ready.get()) {
+      if (System.currentTimeMillis() - start > timeoutMs) {
+        throw new TezException("App Master is not ready within the configured time period (" + timeoutMs + "ms). "
+            + "Please check logs for AM service states.");
+      }
+      try {
+        LOG.info("App is not ready yet, waiting 100ms");
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+        throw new TezException(e);
+      }
+    }
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
index 9fe8e3e72..bb1e6de50 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
@@ -730,6 +730,7 @@ public class TestDAGAppMaster {
   public static class DAGAppMasterForTest extends DAGAppMaster {
     private DAGAppMasterShutdownHandler mockShutdown;
     private TaskSchedulerManager mockScheduler = mock(TaskSchedulerManager.class);
+    private DAGAppMasterReadinessService mockAppMasterReadinessService = mock(DAGAppMasterReadinessService.class);
 
     public DAGAppMasterForTest(ApplicationAttemptId attemptId, boolean isSession) {
       super(attemptId, ContainerId.newContainerId(attemptId, 1), "hostname", 12345, 12346,
@@ -774,5 +775,10 @@ public class TestDAGAppMaster {
         List<NamedEntityDescriptor> taskSchedulerDescriptors) {
       return mockScheduler;
     }
+
+    @Override
+    protected DAGAppMasterReadinessService createAppMasterReadinessService() {
+      return mockAppMasterReadinessService;
+    }
   }
 }