Browse Source

YARN-444. Moved special container exit codes from YarnConfiguration to API where they belong. Contributed by Sandy Ryza.
MAPREDUCE-5151. Updated MR AM to use standard exit codes from the API after YARN-444. Contributed by Sandy Ryza.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1468276 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 12 years ago
parent
commit
0e01f26821
13 changed files with 78 additions and 22 deletions
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  4. 3 0
      hadoop-yarn-project/CHANGES.txt
  5. 44 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerExitStatus.java
  6. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
  7. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  8. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  9. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  10. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  11. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  12. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
  13. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -302,6 +302,9 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5139. Update MR AM to use the modified startContainer API after
     YARN-486. (Xuan Gong via vinodkv)
 
+    MAPREDUCE-5151. Update MR AM to use standard exit codes from the API after
+    YARN-444. (Sandy Ryza via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -67,7 +68,6 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.RackResolver;
 
@@ -624,7 +624,7 @@ public class RMContainerAllocator extends RMContainerRequestor
   @VisibleForTesting
   public TaskAttemptEvent createContainerFinishedEvent(ContainerStatus cont,
       TaskAttemptId attemptID) {
-    if (cont.getExitStatus() == YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS) {
+    if (cont.getExitStatus() == ContainerExitStatus.ABORTED) {
       // killed by framework
       return new TaskAttemptEvent(attemptID,
           TaskAttemptEventType.TA_KILL);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -83,7 +84,6 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
@@ -1660,7 +1660,7 @@ public class TestRMContainerAllocator {
 
     ContainerStatus abortedStatus = BuilderUtils.newContainerStatus(
         containerId, ContainerState.RUNNING, "",
-        YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS);
+        ContainerExitStatus.ABORTED);
     
     TaskAttemptEvent event = allocator.createContainerFinishedEvent(status,
         attemptId);

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -82,6 +82,9 @@ Release 2.0.5-beta - UNRELEASED
     RM as a direct parameter instead of as part of the ContainerLaunchContext
     record. (Xuan Gong via vinodkv)
 
+    YARN-444. Moved special container exit codes from YarnConfiguration to API
+    where they belong. (Sandy Ryza via vinodkv)
+
   NEW FEATURES
 
   IMPROVEMENTS

+ 44 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerExitStatus.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
+*
+*     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.hadoop.yarn.api;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Container exit statuses indicating special exit circumstances.
+ */
+@Public
+@Evolving
+public class ContainerExitStatus {
+  public static final int SUCCESS = 0;
+  public static final int INVALID = -1000;
+
+  /**
+   * Containers killed by the framework, either due to being released by
+   * the application or being 'lost' due to node failures etc.
+   */
+  public static final int ABORTED = -100;
+  
+  /**
+   * When threshold number of the nodemanager-local-directories or
+   * threshold number of the nodemanager-log-directories become bad.
+   */
+  public static final int DISKS_FAILED = -101;
+}

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java

@@ -68,16 +68,17 @@ public interface ContainerStatus {
    *  
    * <p>Note: This is valid only for completed containers i.e. containers
    * with state {@link ContainerState#COMPLETE}. 
-   * Otherwise, it returns an invalid exit code equal to {@literal -1000};</p>
+   * Otherwise, it returns an ContainerExitStatus.INVALID.
+   * </p>
    * 
-   * <p>Container killed by the framework, either due to being released by
+   * <p>Containers killed by the framework, either due to being released by
    * the application or being 'lost' due to node failures etc. have a special
-   * exit code of {@literal -100}.</p>
+   * exit code of ContainerExitStatus.ABORTED.</p>
    * 
    * <p>When threshold number of the nodemanager-local-directories or
    * threshold number of the nodemanager-log-directories become bad, then
-   * container is not launched and is exited with exit status of
-   * {@literal -101}.</p>
+   * container is not launched and is exited with ContainersExitStatus.DISKS_FAILED.
+   * </p>
    *  
    * @return <em>exit status</em> for the container
    */

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -280,7 +280,12 @@ message ContainerStatusProto {
   optional int32 exit_status = 4 [default = -1000];
 }
 
-
+enum ContainerExitStatusProto {
+  SUCCESS = 0;
+  INVALID = -1000;
+  ABORTED = -100;
+  DISKS_FAILED = -101;
+}
 
 ////////////////////////////////////////////////////////////////////////
 ////// From common//////////////////////////////////////////////////////

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.ContainerManager;
 
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -556,7 +557,7 @@ public class ApplicationMaster {
         int exitStatus = containerStatus.getExitStatus();
         if (0 != exitStatus) {
           // container failed
-          if (YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS != exitStatus) {
+          if (ContainerExitStatus.ABORTED != exitStatus) {
             // shell script failed
             // counts as completed
             numCompletedContainers.incrementAndGet();

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -583,11 +583,6 @@ public class YarnConfiguration extends Configuration {
 
   public static final String DEFAULT_NM_USER_HOME_DIR= "/home/";
 
-
-  public static final int INVALID_CONTAINER_EXIT_STATUS = -1000;
-  public static final int ABORTED_CONTAINER_EXIT_STATUS = -100;
-  public static final int DISKS_FAILED = -101;
-
   ////////////////////////////////
   // Web Proxy Configs
   ////////////////////////////////

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -78,7 +79,7 @@ public class ContainerImpl implements Container {
   private final NodeManagerMetrics metrics;
   private final ContainerLaunchContext launchContext;
   private final org.apache.hadoop.yarn.api.records.Container container;
-  private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS;
+  private int exitCode = ContainerExitStatus.INVALID;
   private final StringBuilder diagnostics;
 
   /** The NM-wide configuration - not specific to this container */

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -185,7 +186,7 @@ public class ContainerLaunch implements Callable<Integer> {
       List<String> logDirs = dirsHandler.getLogDirs();
 
       if (!dirsHandler.areDisksHealthy()) {
-        ret = YarnConfiguration.DISKS_FAILED;
+        ret = ContainerExitStatus.DISKS_FAILED;
         throw new IOException("Most of the disks failed. "
             + dirsHandler.getDisksHealthReport());
       }

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java

@@ -26,6 +26,7 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -63,8 +64,9 @@ public class ContainerInfo {
     this.nodeId = nmContext.getNodeId().toString();
     ContainerStatus containerData = container.cloneAndGetContainerStatus();
     this.exitCode = containerData.getExitStatus();
-    this.exitStatus = (this.exitCode == YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS) ? "N/A"
-        : String.valueOf(exitCode);
+    this.exitStatus =
+        (this.exitCode == ContainerExitStatus.INVALID) ?
+            "N/A" : String.valueOf(exitCode);
     this.state = container.getContainerState().toString();
     this.diagnostics = containerData.getDiagnostics();
     if (this.diagnostics == null || this.diagnostics.isEmpty()) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -21,12 +21,12 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceCalculator;
@@ -76,7 +76,7 @@ public class SchedulerUtils {
     containerStatus.setContainerId(containerId);
     containerStatus.setDiagnostics(diagnostics);
     containerStatus.setExitStatus(
-        YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS);
+        ContainerExitStatus.ABORTED);
     containerStatus.setState(ContainerState.COMPLETE);
     return containerStatus;
   }