You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by as...@apache.org on 2015/12/30 07:56:41 UTC

[16/27] hadoop git commit: YARN-2882. Add an OPPORTUNISTIC ExecutionType. (Konstantinos Karanasos and Inigo Goiri via kasha)

YARN-2882. Add an OPPORTUNISTIC ExecutionType. (Konstantinos Karanasos and Inigo Goiri via kasha)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fb007943
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fb007943
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fb007943

Branch: refs/heads/yarn-2877
Commit: fb00794368e0aa7aafa9dfc8d453810f641b82b2
Parents: 8cfd672
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Sat Dec 26 20:22:16 2015 -0800
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Sat Dec 26 20:22:16 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../yarn/api/records/ContainerStatus.java       | 24 +++++++++++
 .../hadoop/yarn/api/records/ExecutionType.java  | 43 ++++++++++++++++++++
 .../yarn/server/api/ContainerContext.java       | 22 ++++++++++
 .../src/main/proto/yarn_protos.proto            |  6 +++
 .../records/impl/pb/ContainerStatusPBImpl.java  | 31 +++++++++++++-
 .../yarn/api/records/impl/pb/ProtoUtils.java    | 12 ++++++
 .../yarn/security/ContainerTokenIdentifier.java | 35 +++++++++++++++-
 .../src/main/proto/yarn_security_token.proto    |  1 +
 .../yarn/security/TestYARNTokenIdentifier.java  | 13 +++++-
 10 files changed, 187 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 44c8ccf..e380893 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -6,6 +6,9 @@ Trunk - Unreleased
 
   NEW FEATURES
 
+    YARN-2882. Add an OPPORTUNISTIC ExecutionType. 
+    (Konstantinos Karanasos and Inigo Goiri via kasha)
+
   IMPROVEMENTS
 
     YARN-2438. yarn-env.sh cleanup (aw)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 2c2238f..4f56535 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.api.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
@@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.util.Records;
  * It provides details such as:
  * <ul>
  *   <li>{@code ContainerId} of the container.</li>
+ *   <li>{@code ExecutionType} of the container.</li>
  *   <li>{@code ContainerState} of the container.</li>
  *   <li><em>Exit status</em> of a completed container.</li>
  *   <li><em>Diagnostic</em> message for a failed container.</li>
@@ -45,7 +47,17 @@ public abstract class ContainerStatus {
   @Unstable
   public static ContainerStatus newInstance(ContainerId containerId,
       ContainerState containerState, String diagnostics, int exitStatus) {
+    return newInstance(containerId, ExecutionType.GUARANTEED, containerState,
+        diagnostics, exitStatus);
+  }
+
+  @Private
+  @Unstable
+  public static ContainerStatus newInstance(ContainerId containerId,
+      ExecutionType executionType, ContainerState containerState,
+      String diagnostics, int exitStatus) {
     ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class);
+    containerStatus.setExecutionType(executionType);
     containerStatus.setState(containerState);
     containerStatus.setContainerId(containerId);
     containerStatus.setDiagnostics(diagnostics);
@@ -66,6 +78,18 @@ public abstract class ContainerStatus {
   public abstract void setContainerId(ContainerId containerId);
 
   /**
+   * Get the <code>ExecutionType</code> of the container.
+   * @return <code>ExecutionType</code> of the container
+   */
+  @Public
+  @Evolving
+  public abstract ExecutionType getExecutionType();
+
+  @Private
+  @Unstable
+  public abstract void setExecutionType(ExecutionType executionType);
+
+  /**
    * Get the <code>ContainerState</code> of the container.
    * @return <code>ContainerState</code> of the container
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionType.java
new file mode 100644
index 0000000..27cc74d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ExecutionType.java
@@ -0,0 +1,43 @@
+/**
+ * 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.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Container property encoding execution semantics.
+ *
+ * <p>
+ * The execution types are the following:
+ * <ul>
+ *   <li>{@link #GUARANTEED} - this container is guaranteed to start its
+ *   execution, once the corresponding start container request is received by
+ *   an NM.
+ *   <li>{@link #OPPORTUNISTIC} - the execution of this container may not start
+ *   immediately at the NM that receives the corresponding start container
+ *   request (depending on the NM's available resources). Moreover, it may be
+ *   preempted if it blocks a GUARANTEED container from being executed.
+ * </ul>
+ */
+@Public
+@Evolving
+public enum ExecutionType {
+  GUARANTEED, OPPORTUNISTIC
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
index f7a9b02..fbf3345 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerContext.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 
 /**
@@ -36,6 +37,7 @@ public class ContainerContext {
   private final ContainerId containerId;
   private final Resource resource;
   private final ContainerType containerType;
+  private final ExecutionType executionType;
 
   @Private
   @Unstable
@@ -48,10 +50,20 @@ public class ContainerContext {
   @Unstable
   public ContainerContext(String user, ContainerId containerId,
       Resource resource, ContainerType containerType) {
+    this(user, containerId, resource, containerType,
+        ExecutionType.GUARANTEED);
+  }
+
+  @Private
+  @Unstable
+  public ContainerContext(String user, ContainerId containerId,
+      Resource resource, ContainerType containerType,
+      ExecutionType executionType) {
     this.user = user;
     this.containerId = containerId;
     this.resource = resource;
     this.containerType = containerType;
+    this.executionType = executionType;
   }
 
   /**
@@ -91,4 +103,14 @@ public class ContainerContext {
   public ContainerType getContainerType() {
     return containerType;
   }
+
+  /**
+   * Get {@link ExecutionType} the execution type of the container
+   * being initialized or stopped.
+   *
+   * @return the execution type of the container
+   */
+  public ExecutionType getExecutionType() {
+    return executionType;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 3c208e2..d122f5a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -284,6 +284,11 @@ enum ContainerTypeProto {
   TASK = 2;
 }
 
+enum ExecutionTypeProto {
+  GUARANTEED = 1;
+  OPPORTUNISTIC = 2;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
@@ -490,6 +495,7 @@ message ContainerStatusProto {
   optional string diagnostics = 3 [default = "N/A"];
   optional int32 exit_status = 4 [default = -1000];
   optional ResourceProto capability = 5;
+  optional ExecutionTypeProto executionType = 6 [default = GUARANTEED];
 }
 
 enum ContainerExitStatusProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
index d33d06d..f1fdae9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
@@ -24,9 +24,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProtoOrBuilder;
@@ -79,6 +81,7 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     StringBuilder sb = new StringBuilder();
     sb.append("ContainerStatus: [");
     sb.append("ContainerId: ").append(getContainerId()).append(", ");
+    sb.append("ExecutionType: ").append(getExecutionType()).append(", ");
     sb.append("State: ").append(getState()).append(", ");
     sb.append("Capability: ").append(getCapability()).append(", ");
     sb.append("Diagnostics: ").append(getDiagnostics()).append(", ");
@@ -107,7 +110,25 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     }
     viaProto = false;
   }
-    
+
+  @Override
+  public synchronized ExecutionType getExecutionType() {
+    ContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasExecutionType()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getExecutionType());
+  }
+
+  @Override
+  public synchronized void setExecutionType(ExecutionType executionType) {
+    maybeInitBuilder();
+    if (executionType == null) {
+      builder.clearExecutionType();
+      return;
+    }
+    builder.setExecutionType(convertToProtoFormat(executionType));
+  }
   
   @Override
   public synchronized ContainerState getState() {
@@ -206,6 +227,14 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     return ((ContainerIdPBImpl)t).getProto();
   }
 
+  private ExecutionType convertFromProtoFormat(ExecutionTypeProto e) {
+    return ProtoUtils.convertFromProtoFormat(e);
+  }
+
+  private ExecutionTypeProto convertToProtoFormat(ExecutionType e) {
+    return ProtoUtils.convertToProtoFormat(e);
+  }
+
   private ResourceProto convertToProtoFormat(Resource e) {
     return ((ResourcePBImpl)e).getProto();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index e742f4c..29ed0f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -54,6 +55,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProt
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
@@ -282,4 +284,14 @@ public class ProtoUtils {
   public static ContainerType convertFromProtoFormat(ContainerTypeProto e) {
     return ContainerType.valueOf(e.name());
   }
+
+  /*
+   * ExecutionType
+   */
+  public static ExecutionTypeProto convertToProtoFormat(ExecutionType e) {
+    return ExecutionTypeProto.valueOf(e.name());
+  }
+  public static ExecutionType convertFromProtoFormat(ExecutionTypeProto e) {
+    return ExecutionType.valueOf(e.name());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
index 106e6d5..0b03f50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeProto;
 import org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
@@ -85,6 +87,16 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       long rmIdentifier, Priority priority, long creationTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression,
       ContainerType containerType) {
+    this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
+        rmIdentifier, priority, creationTime, logAggregationContext,
+        nodeLabelExpression, containerType, ExecutionType.GUARANTEED);
+  }
+
+  public ContainerTokenIdentifier(ContainerId containerID, String hostName,
+      String appSubmitter, Resource r, long expiryTimeStamp, int masterKeyId,
+      long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType, ExecutionType executionType) {
     ContainerTokenIdentifierProto.Builder builder =
         ContainerTokenIdentifierProto.newBuilder();
     if (containerID != null) {
@@ -112,6 +124,7 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       builder.setNodeLabelExpression(nodeLabelExpression);
     }
     builder.setContainerType(convertToProtoFormat(containerType));
+    builder.setExecutionType(convertToProtoFormat(executionType));
 
     proto = builder.build();
   }
@@ -163,7 +176,7 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     return proto.getCreationTime();
   }
   /**
-   * Get the RMIdentifier of RM in which containers are allocated
+   * Get the RMIdentifier of RM in which containers are allocated.
    * @return RMIdentifier
    */
   public long getRMIdentifier() {
@@ -181,6 +194,17 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     return convertFromProtoFormat(proto.getContainerType());
   }
 
+  /**
+   * Get the ExecutionType of container to allocate
+   * @return ExecutionType
+   */
+  public ExecutionType getExecutionType(){
+    if (!proto.hasExecutionType()) {
+      return null;
+    }
+    return convertFromProtoFormat(proto.getExecutionType());
+  }
+
   public ContainerTokenIdentifierProto getProto() {
     return proto;
   }
@@ -265,4 +289,13 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       ContainerTypeProto containerType) {
     return ProtoUtils.convertFromProtoFormat(containerType);
   }
+
+  private ExecutionTypeProto convertToProtoFormat(ExecutionType executionType) {
+    return ProtoUtils.convertToProtoFormat(executionType);
+  }
+
+  private ExecutionType convertFromProtoFormat(
+      ExecutionTypeProto executionType) {
+    return ProtoUtils.convertFromProtoFormat(executionType);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto
index 339e99e..71434be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto
@@ -51,6 +51,7 @@ message ContainerTokenIdentifierProto {
   optional LogAggregationContextProto logAggregationContext = 10;
   optional string nodeLabelExpression = 11;
   optional ContainerTypeProto containerType = 12;
+  optional ExecutionTypeProto executionType = 13 [default = GUARANTEED];
 }
 
 message ClientToAMTokenIdentifierProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb007943/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
index 68f0b9d..3a5f003 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.security.HadoopKerberosName;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -209,6 +210,9 @@ public class TestYARNTokenIdentifier {
 
     Assert.assertEquals(ContainerType.TASK,
         anotherToken.getContainerType());
+
+    Assert.assertEquals(ExecutionType.GUARANTEED,
+        anotherToken.getExecutionType());
   }
   
   @Test
@@ -384,10 +388,14 @@ public class TestYARNTokenIdentifier {
     Assert.assertEquals(ContainerType.APPLICATION_MASTER,
         anotherToken.getContainerType());
 
+    Assert.assertEquals(ExecutionType.GUARANTEED,
+        anotherToken.getExecutionType());
+
     token =
         new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
             expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
-            null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
+            null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
+            ExecutionType.OPPORTUNISTIC);
 
     anotherToken = new ContainerTokenIdentifier();
 
@@ -398,6 +406,9 @@ public class TestYARNTokenIdentifier {
 
     Assert.assertEquals(ContainerType.TASK,
         anotherToken.getContainerType());
+
+    Assert.assertEquals(ExecutionType.OPPORTUNISTIC,
+        anotherToken.getExecutionType());
   }
 
 }