diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
index 5c4156b..cd85495 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
@@ -112,6 +112,7 @@
yarn_protos.proto
yarn_service_protos.proto
applicationmaster_protocol.proto
+ distributed_scheduler_protocol.proto
applicationclient_protocol.proto
containermanagement_protocol.proto
server/yarn_server_resourcemanager_service_protos.proto
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocol.java
new file mode 100644
index 0000000..f47bf73
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocol.java
@@ -0,0 +1,46 @@
+/**
+ * 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.Unstable;
+import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import java.io.IOException;
+
+public interface DistributedSchedulerProtocol extends ApplicationMasterProtocol {
+
+ @Public
+ @Unstable
+ @Idempotent
+ DistSchedRegisterResponse registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request)
+ throws YarnException, IOException;
+
+ @Public
+ @Unstable
+ @Idempotent
+ DistSchedAllocateResponse allocateForDistributedScheduling(
+ AllocateRequest request) throws YarnException, IOException;
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedAllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedAllocateResponse.java
new file mode 100644
index 0000000..c31c764
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedAllocateResponse.java
@@ -0,0 +1,57 @@
+/**
+ * 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.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+@Public
+@Unstable
+public abstract class DistSchedAllocateResponse {
+
+ @Public
+ @Unstable
+ public static DistSchedAllocateResponse newInstance(AllocateResponse
+ allResp) {
+ DistSchedAllocateResponse response =
+ Records.newRecord(DistSchedAllocateResponse.class);
+ response.setAllocateResponse(allResp);
+ return response;
+ }
+
+ @Public
+ @Unstable
+ public abstract void setAllocateResponse(AllocateResponse response);
+
+ @Public
+ @Unstable
+ public abstract AllocateResponse getAllocateResponse();
+
+ @Public
+ @Unstable
+ public abstract void setNodesForScheduling(List nodesForScheduling);
+
+ @Public
+ @Unstable
+ public abstract List getNodesForScheduling();
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedRegisterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedRegisterResponse.java
new file mode 100644
index 0000000..8a79c4c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/DistSchedRegisterResponse.java
@@ -0,0 +1,92 @@
+/**
+ * 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.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+@Public
+@Unstable
+public abstract class DistSchedRegisterResponse {
+
+ @Public
+ @Unstable
+ public static DistSchedRegisterResponse newInstance
+ (RegisterApplicationMasterResponse regAMResp) {
+ DistSchedRegisterResponse response =
+ Records.newRecord(DistSchedRegisterResponse.class);
+ response.setRegisterResponse(regAMResp);
+ return response;
+ }
+
+ @Public
+ @Unstable
+ public abstract void setRegisterResponse(
+ RegisterApplicationMasterResponse resp);
+
+ @Public
+ @Unstable
+ public abstract RegisterApplicationMasterResponse getRegisterResponse();
+
+ @Public
+ @Unstable
+ public abstract void setMinAllocatableCapabilty(Resource minResource);
+
+ @Public
+ @Unstable
+ public abstract Resource getMinAllocatableCapabilty();
+
+ @Public
+ @Unstable
+ public abstract void setMaxAllocatableCapabilty(Resource maxResource);
+
+ @Public
+ @Unstable
+ public abstract Resource getMaxAllocatableCapabilty();
+
+ @Public
+ @Unstable
+ public abstract void setContainerTokenExpiryInterval(int interval);
+
+ @Public
+ @Unstable
+ public abstract int getContainerTokenExpiryInterval();
+
+ @Public
+ @Unstable
+ public abstract void setContainerIdStart(long containerIdStart);
+
+ @Public
+ @Unstable
+ public abstract long getContainerIdStart();
+
+ @Public
+ @Unstable
+ public abstract void setNodesForScheduling(List nodesForScheduling);
+
+ @Public
+ @Unstable
+ public abstract List getNodesForScheduling();
+
+}
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..96d510c 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
@@ -31,6 +31,7 @@
* It provides details such as:
*
* - {@code ContainerId} of the container.
+ * - {@code ExecutionType} of the container.
* - {@code ContainerState} of the container.
* - Exit status of a completed container.
* - Diagnostic message for a failed container.
@@ -45,7 +46,17 @@
@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 +77,18 @@ public static ContainerStatus newInstance(ContainerId containerId,
public abstract void setContainerId(ContainerId containerId);
/**
+ * Get the ExecutionType of the container.
+ * @return ExecutionType of the container
+ */
+ @Public
+ @Stable
+ public abstract ExecutionType getExecutionType();
+
+ @Private
+ @Unstable
+ public abstract void setExecutionType(ExecutionType executionType);
+
+ /**
* Get the ContainerState of the container.
* @return ContainerState of the container
*/
@@ -76,7 +99,7 @@ public static ContainerStatus newInstance(ContainerId containerId,
@Private
@Unstable
public abstract void setState(ContainerState state);
-
+
/**
* Get the exit status for the container.
*
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..c36f4b0
--- /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,39 @@
+/**
+ * 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;
+
+/**
+ * Container and resource request property encoding execution semantics.
+ *
+ *
+ * The execution types are the following:
+ *
+ * - {@link #GUARANTEED} - this container is guaranteed to start its
+ * execution, once received by an NM.
+ * - {@link #QUEUEABLE} - this container is allowed to be queued in one of the
+ * NMs. It is thus not guaranteed to start its execution immediately.
+ *
+ *
+ */
+@Public
+public enum ExecutionType {
+ GUARANTEED, QUEUEABLE
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
index 790120a..5729e18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
@@ -79,6 +79,15 @@ public static ResourceRequest newInstance(Priority priority, String hostName,
public static ResourceRequest newInstance(Priority priority, String hostName,
Resource capability, int numContainers, boolean relaxLocality,
String labelExpression) {
+ return newInstance(priority, hostName, capability, numContainers,
+ relaxLocality, labelExpression, ExecutionType.GUARANTEED);
+ }
+
+ @Public
+ @Stable
+ public static ResourceRequest newInstance(Priority priority, String hostName,
+ Resource capability, int numContainers, boolean relaxLocality, String
+ labelExpression, ExecutionType execType) {
ResourceRequest request = Records.newRecord(ResourceRequest.class);
request.setPriority(priority);
request.setResourceName(hostName);
@@ -86,6 +95,7 @@ public static ResourceRequest newInstance(Priority priority, String hostName,
request.setNumContainers(numContainers);
request.setRelaxLocality(relaxLocality);
request.setNodeLabelExpression(labelExpression);
+ request.setExecutionType(execType);
return request;
}
@@ -221,7 +231,26 @@ public static boolean isAnyLocation(String hostName) {
@Public
@Stable
public abstract boolean getRelaxLocality();
-
+
+ /**
+ * Set the ExecutionType of the requested container
+ * @param execType ExecutionType of the
+ */
+ @Public
+ @Stable
+ public abstract void setExecutionType(ExecutionType execType);
+
+ /**
+ * Get whether locality relaxation is enabled with this
+ * ResourceRequest. Defaults to true.
+ *
+ * @return whether locality relaxation is enabled with this
+ * ResourceRequest.
+ */
+ @Public
+ @Stable
+ public abstract ExecutionType getExecutionType();
+
/**
* For a request at a network hierarchy level, set whether locality can be relaxed
* to that level and beyond.
@@ -322,6 +351,12 @@ public boolean equals(Object obj) {
return false;
} else if (!priority.equals(other.getPriority()))
return false;
+ ExecutionType executionType = getExecutionType();
+ if (executionType == null) {
+ if (other.getExecutionType() != null)
+ return false;
+ } else if (executionType != other.getExecutionType())
+ return false;
if (getNodeLabelExpression() == null) {
if (other.getNodeLabelExpression() != null) {
return false;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index cbd28ca..18fc76f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -282,6 +282,38 @@ private static void addDeprecatedKeys() {
/** ACL used in case none is found. Allows nothing. */
public static final String DEFAULT_YARN_APP_ACL = " ";
+ /** Is Distributed Scheduling Enabled */
+ public static String DIST_SCHEDULING_ENABLED =
+ YARN_PREFIX + "distributed-scheduling.enabled";
+ public static final boolean DIST_SCHEDULING_ENABLED_DEFAULT = false;
+
+ /** Mininum allocatable container memory for Distributed Scheduling */
+ public static String DIST_SCHEDULING_MIN_MEMORY =
+ YARN_PREFIX + "distributed-scheduling.min-memory";
+ public static final int DIST_SCHEDULING_MIN_MEMORY_DEFAULT = 512;
+
+ /** Mininum allocatable container vcores for Distributed Scheduling */
+ public static String DIST_SCHEDULING_MIN_VCORES =
+ YARN_PREFIX + "distributed-scheduling.min-vcores";
+ public static final int DIST_SCHEDULING_MIN_VCORES_DEFAULT = 1;
+
+ /** Maximum allocatable container memory for Distributed Scheduling */
+ public static String DIST_SCHEDULING_MAX_MEMORY =
+ YARN_PREFIX + "distributed-scheduling.max-memory";
+ public static final int DIST_SCHEDULING_MAX_MEMORY_DEFAULT = 2048;
+
+ /** Maximum allocatable container vcores for Distributed Scheduling */
+ public static String DIST_SCHEDULING_MAX_VCORES =
+ YARN_PREFIX + "distributed-scheduling.max-vcores";
+ public static final int DIST_SCHEDULING_MAX_VCORES_DEFAULT = 4;
+
+ /** Container token expiry for container allocated via
+ * Distributed Scheduling */
+ public static String DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS =
+ YARN_PREFIX + "distributed-scheduling.container-token-expiry";
+ public static final int DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS_DEFAULT =
+ 600000;
+
/**
* Enable/disable intermediate-data encryption at YARN level. For now, this
* only is used by the FileSystemRMStateStore to setup right file-system
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..74be35f 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.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,22 +37,24 @@
private final ContainerId containerId;
private final Resource resource;
private final ContainerType containerType;
+ private final ExecutionType executionType;
@Private
@Unstable
public ContainerContext(String user, ContainerId containerId,
Resource resource) {
- this(user, containerId, resource, ContainerType.TASK);
+ this(user, containerId, resource, ContainerType.TASK, ExecutionType.GUARANTEED);
}
@Private
@Unstable
public ContainerContext(String user, ContainerId containerId,
- Resource resource, ContainerType containerType) {
+ Resource resource, ContainerType containerType, ExecutionType executionType) {
this.user = user;
this.containerId = containerId;
this.resource = resource;
this.containerType = containerType;
+ this.executionType = executionType;
}
/**
@@ -91,4 +94,14 @@ public Resource getResource() {
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;
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
index 44428f9..6256c89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerInitializationContext.java
@@ -23,6 +23,7 @@
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;
/**
@@ -44,8 +45,8 @@ public ContainerInitializationContext(String user, ContainerId containerId,
@Private
@Unstable
public ContainerInitializationContext(String user, ContainerId containerId,
- Resource resource, ContainerType containerType) {
- super(user, containerId, resource, containerType);
+ Resource resource, ContainerType containerType, ExecutionType executionType) {
+ super(user, containerId, resource, containerType, executionType);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
index 2e4ad3b..1b8f18f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ContainerTerminationContext.java
@@ -23,6 +23,7 @@
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;
/**
@@ -44,8 +45,8 @@ public ContainerTerminationContext(String user, ContainerId containerId,
@Private
@Unstable
public ContainerTerminationContext(String user, ContainerId containerId,
- Resource resource, ContainerType containerType) {
- super(user, containerId, resource, containerType);
+ Resource resource, ContainerType containerType, ExecutionType executionType) {
+ super(user, containerId, resource, containerType, executionType);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/distributed_scheduler_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/distributed_scheduler_protocol.proto
new file mode 100644
index 0000000..fb66d21
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/distributed_scheduler_protocol.proto
@@ -0,0 +1,37 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are public and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.yarn.proto";
+option java_outer_classname = "DistributedSchedulerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.yarn;
+
+import "yarn_service_protos.proto";
+
+
+service DistributedSchedulerProtocolService {
+ rpc registerApplicationMasterForDistributedScheduling (RegisterApplicationMasterRequestProto) returns (DistSchedRegisterResponseProto);
+ rpc allocateForDistributedScheduling (AllocateRequestProto) returns (DistSchedAllocateResponseProto);
+}
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 09d2bd5..f6b7466 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
@@ -275,6 +275,11 @@ enum ContainerTypeProto {
TASK = 2;
}
+enum ExecutionTypeProto {
+ GUARANTEED = 1;
+ QUEUEABLE = 2;
+}
+
////////////////////////////////////////////////////////////////////////
////// From AM_RM_Protocol /////////////////////////////////////////////
////////////////////////////////////////////////////////////////////////
@@ -285,6 +290,7 @@ message ResourceRequestProto {
optional int32 num_containers = 4;
optional bool relax_locality = 5 [default = true];
optional string node_label_expression = 6;
+ optional ExecutionTypeProto executionType = 7 [default = GUARANTEED];
}
enum AMCommandProto {
@@ -480,6 +486,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 {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 8924eba..2f9eec2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -50,6 +50,15 @@ message RegisterApplicationMasterResponseProto {
repeated SchedulerResourceTypes scheduler_resource_types = 7;
}
+message DistSchedRegisterResponseProto {
+ optional RegisterApplicationMasterResponseProto register_response = 1;
+ optional ResourceProto max_alloc_capability = 2;
+ optional ResourceProto min_alloc_capability = 3;
+ optional int32 container_token_expiry_interval = 4;
+ optional int64 container_id_start = 5;
+ repeated NodeIdProto nodes_for_scheduling = 6;
+}
+
message FinishApplicationMasterRequestProto {
optional string diagnostics = 1;
optional string tracking_url = 2;
@@ -91,6 +100,11 @@ message AllocateResponseProto {
optional PriorityProto application_priority = 13;
}
+message DistSchedAllocateResponseProto {
+ optional AllocateResponseProto allocate_response = 1;
+ repeated NodeIdProto nodes_for_scheduling = 2;
+}
+
enum SchedulerResourceTypes {
MEMORY = 0;
CPU = 1;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index e0bf2d3..c0ed4cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -34,6 +34,7 @@
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
@@ -108,6 +109,7 @@ protected AMRMClient(String name) {
final Priority priority;
final boolean relaxLocality;
final String nodeLabelsExpression;
+ final ExecutionType executionType;
/**
* Instantiates a {@link ContainerRequest} with the given constraints and
@@ -152,6 +154,33 @@ public ContainerRequest(Resource capability, String[] nodes,
String[] racks, Priority priority, boolean relaxLocality) {
this(capability, nodes, racks, priority, relaxLocality, null);
}
+
+ /**
+ * Instantiates a {@link ContainerRequest} with the given constraints.
+ *
+ * @param capability
+ * The {@link Resource} to be requested for each container.
+ * @param nodes
+ * Any hosts to request that the containers are placed on.
+ * @param racks
+ * Any racks to request that the containers are placed on. The
+ * racks corresponding to any hosts requested will be automatically
+ * added to this list.
+ * @param priority
+ * The priority at which to request the containers. Higher
+ * priorities have lower numerical values.
+ * @param relaxLocality
+ * If true, containers for this request may be assigned on hosts
+ * and racks other than the ones explicitly requested.
+ * @param nodeLabelsExpression
+ * Set node labels to allocate resource, now we only support
+ * asking for only a single node label
+ */
+ public ContainerRequest(Resource capability, String[] nodes, String[] racks,
+ Priority priority, boolean relaxLocality, String nodeLabelsExpression) {
+ this(capability, nodes, racks, priority, relaxLocality, null,
+ ExecutionType.GUARANTEED);
+ }
/**
* Instantiates a {@link ContainerRequest} with the given constraints.
@@ -173,10 +202,12 @@ public ContainerRequest(Resource capability, String[] nodes,
* @param nodeLabelsExpression
* Set node labels to allocate resource, now we only support
* asking for only a single node label
+ * @param executionType
+ * Set the execution type of the container request.
*/
- public ContainerRequest(Resource capability, String[] nodes,
- String[] racks, Priority priority, boolean relaxLocality,
- String nodeLabelsExpression) {
+ public ContainerRequest(Resource capability, String[] nodes, String[] racks,
+ Priority priority, boolean relaxLocality, String nodeLabelsExpression,
+ ExecutionType executionType) {
// Validate request
Preconditions.checkArgument(capability != null,
"The Resource to be requested for each container " +
@@ -194,6 +225,7 @@ public ContainerRequest(Resource capability, String[] nodes,
this.priority = priority;
this.relaxLocality = relaxLocality;
this.nodeLabelsExpression = nodeLabelsExpression;
+ this.executionType = executionType;
}
public Resource getCapability() {
@@ -220,10 +252,15 @@ public String getNodeLabelExpression() {
return nodeLabelsExpression;
}
+ public ExecutionType getExecutionType() {
+ return executionType;
+ }
+
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("Capability[").append(capability).append("]");
sb.append("Priority[").append(priority).append("]");
+ sb.append("ExecutionType[").append(executionType).append("]");
return sb.toString();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocolPB.java
new file mode 100644
index 0000000..b750a60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/DistributedSchedulerProtocolPB.java
@@ -0,0 +1,34 @@
+/**
+ * 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.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
+import org.apache.hadoop.yarn.proto.DistributedSchedulerProtocol.DistributedSchedulerProtocolService;
+
+@Private
+@Unstable
+@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.DistributedSchedulerProtocolPB",
+ protocolVersion = 1)
+public interface DistributedSchedulerProtocolPB extends
+ DistributedSchedulerProtocolService.BlockingInterface,
+ ApplicationMasterProtocolService.BlockingInterface {
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java
new file mode 100644
index 0000000..688f99a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/DistributedSchedulerProtocolPBClientImpl.java
@@ -0,0 +1,153 @@
+/**
+ * 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.impl.pb.client;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+
+
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .DistSchedAllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .DistSchedRegisterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+public class DistributedSchedulerProtocolPBClientImpl implements
+ DistributedSchedulerProtocol, Closeable {
+
+ private DistributedSchedulerProtocolPB proxy;
+
+ public DistributedSchedulerProtocolPBClientImpl(long clientVersion,
+ InetSocketAddress addr,
+ Configuration conf) throws IOException {
+ RPC.setProtocolEngine(conf, DistributedSchedulerProtocolPB.class,
+ ProtobufRpcEngine.class);
+ proxy = RPC.getProxy(DistributedSchedulerProtocolPB.class, clientVersion,
+ addr, conf);
+ }
+
+ @Override
+ public void close() {
+ if (this.proxy != null) {
+ RPC.stopProxy(this.proxy);
+ }
+ }
+
+ @Override
+ public DistSchedRegisterResponse
+ registerApplicationMasterForDistributedScheduling
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
+ ((RegisterApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new DistSchedRegisterResponsePBImpl(
+ proxy.registerApplicationMasterForDistributedScheduling(
+ null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public DistSchedAllocateResponse allocateForDistributedScheduling
+ (AllocateRequest request) throws YarnException, IOException {
+ YarnServiceProtos.AllocateRequestProto requestProto =
+ ((AllocateRequestPBImpl) request).getProto();
+ try {
+ return new DistSchedAllocateResponsePBImpl(
+ proxy.allocateForDistributedScheduling(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ YarnServiceProtos.RegisterApplicationMasterRequestProto requestProto =
+ ((RegisterApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new RegisterApplicationMasterResponsePBImpl(
+ proxy.registerApplicationMaster(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ YarnServiceProtos.FinishApplicationMasterRequestProto requestProto =
+ ((FinishApplicationMasterRequestPBImpl) request).getProto();
+ try {
+ return new FinishApplicationMasterResponsePBImpl(
+ proxy.finishApplicationMaster(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ YarnServiceProtos.AllocateRequestProto requestProto =
+ ((AllocateRequestPBImpl) request).getProto();
+ try {
+ return new AllocateResponsePBImpl(proxy.allocate(null, requestProto));
+ } catch (ServiceException e) {
+ RPCUtil.unwrapAndThrowException(e);
+ return null;
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java
new file mode 100644
index 0000000..44cc54f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/DistributedSchedulerProtocolPBServiceImpl.java
@@ -0,0 +1,144 @@
+/**
+ * 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.impl.pb.service;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.DistSchedAllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.DistSchedRegisterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.DistSchedAllocateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.DistSchedRegisterResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
+
+import java.io.IOException;
+
+public class DistributedSchedulerProtocolPBServiceImpl implements
+ DistributedSchedulerProtocolPB {
+
+ private DistributedSchedulerProtocol real;
+
+ public DistributedSchedulerProtocolPBServiceImpl(
+ DistributedSchedulerProtocol impl) {
+ this.real = impl;
+ }
+
+ @Override
+ public DistSchedRegisterResponseProto
+ registerApplicationMasterForDistributedScheduling(RpcController controller,
+ RegisterApplicationMasterRequestProto proto) throws
+ ServiceException {
+ RegisterApplicationMasterRequestPBImpl request = new
+ RegisterApplicationMasterRequestPBImpl(proto);
+ try {
+ DistSchedRegisterResponse response =
+ real.registerApplicationMasterForDistributedScheduling(request);
+ return ((DistSchedRegisterResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public DistSchedAllocateResponseProto
+ allocateForDistributedScheduling(RpcController controller,
+ AllocateRequestProto proto) throws ServiceException {
+ AllocateRequestPBImpl request = new AllocateRequestPBImpl(proto);
+ try {
+ DistSchedAllocateResponse response = real
+ .allocateForDistributedScheduling(request);
+ return ((DistSchedAllocateResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.AllocateResponseProto allocate(RpcController arg0,
+ AllocateRequestProto proto) throws ServiceException {
+ AllocateRequestPBImpl request = new AllocateRequestPBImpl(proto);
+ try {
+ AllocateResponse response = real.allocate(request);
+ return ((AllocateResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.FinishApplicationMasterResponseProto
+ finishApplicationMaster(
+ RpcController arg0, YarnServiceProtos
+ .FinishApplicationMasterRequestProto proto)
+ throws ServiceException {
+ FinishApplicationMasterRequestPBImpl request = new
+ FinishApplicationMasterRequestPBImpl(proto);
+ try {
+ FinishApplicationMasterResponse response = real.finishApplicationMaster
+ (request);
+ return ((FinishApplicationMasterResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public YarnServiceProtos.RegisterApplicationMasterResponseProto
+ registerApplicationMaster(
+ RpcController arg0, RegisterApplicationMasterRequestProto proto)
+ throws ServiceException {
+ RegisterApplicationMasterRequestPBImpl request = new
+ RegisterApplicationMasterRequestPBImpl(proto);
+ try {
+ RegisterApplicationMasterResponse response = real
+ .registerApplicationMaster(request);
+ return ((RegisterApplicationMasterResponsePBImpl) response).getProto();
+ } catch (YarnException e) {
+ throw new ServiceException(e);
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java
new file mode 100644
index 0000000..f97bcef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedAllocateResponsePBImpl.java
@@ -0,0 +1,175 @@
+/**
+ * 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.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.DistSchedAllocateResponseProto;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class DistSchedAllocateResponsePBImpl extends DistSchedAllocateResponse {
+
+ DistSchedAllocateResponseProto proto =
+ DistSchedAllocateResponseProto.getDefaultInstance();
+ DistSchedAllocateResponseProto.Builder builder = null;
+ boolean viaProto = false;
+
+ private AllocateResponse allocateResponse;
+ private List nodesForScheduling;
+
+ public DistSchedAllocateResponsePBImpl() {
+ builder = DistSchedAllocateResponseProto.newBuilder();
+ }
+
+ public DistSchedAllocateResponsePBImpl(DistSchedAllocateResponseProto proto) {
+ this.proto = proto;
+ viaProto = true;
+ }
+
+ public DistSchedAllocateResponseProto getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
+ private void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = DistSchedAllocateResponseProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
+ private synchronized void mergeLocalToProto() {
+ if (viaProto)
+ maybeInitBuilder();
+ mergeLocalToBuilder();
+ proto = builder.build();
+ viaProto = true;
+ }
+
+ private synchronized void mergeLocalToBuilder() {
+ if (this.nodesForScheduling != null) {
+ builder.clearNodesForScheduling();
+ Iterable iterable =
+ getNodeIdProtoIterable(this.nodesForScheduling);
+ builder.addAllNodesForScheduling(iterable);
+ }
+ if (this.allocateResponse != null) {
+ builder.setAllocateResponse(
+ ((AllocateResponsePBImpl)this.allocateResponse).getProto());
+ }
+ }
+ @Override
+ public void setAllocateResponse(AllocateResponse response) {
+ maybeInitBuilder();
+ if(allocateResponse == null) {
+ builder.clearAllocateResponse();
+ }
+ this.allocateResponse = response;
+ }
+
+ @Override
+ public AllocateResponse getAllocateResponse() {
+ if (this.allocateResponse != null) {
+ return this.allocateResponse;
+ }
+
+ YarnServiceProtos.DistSchedAllocateResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ if (!p.hasAllocateResponse()) {
+ return null;
+ }
+
+ this.allocateResponse =
+ new AllocateResponsePBImpl(p.getAllocateResponse());
+ return this.allocateResponse;
+ }
+
+ @Override
+ public void setNodesForScheduling(List nodesForScheduling) {
+ maybeInitBuilder();
+ if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
+ if (this.nodesForScheduling != null) {
+ this.nodesForScheduling.clear();
+ }
+ builder.clearNodesForScheduling();
+ return;
+ }
+ this.nodesForScheduling = new ArrayList<>();
+ this.nodesForScheduling.addAll(nodesForScheduling);
+ }
+
+ @Override
+ public List getNodesForScheduling() {
+ if (nodesForScheduling != null) {
+ return nodesForScheduling;
+ }
+ initLocalNodesForSchedulingList();
+ return nodesForScheduling;
+ }
+
+ private synchronized void initLocalNodesForSchedulingList() {
+ YarnServiceProtos.DistSchedAllocateResponseProtoOrBuilder p =
+ viaProto ? proto : builder;
+ List list = p.getNodesForSchedulingList();
+ nodesForScheduling = new ArrayList<>();
+ if (list != null) {
+ for (YarnProtos.NodeIdProto t : list) {
+ nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
+ }
+ }
+ }
+ private synchronized Iterable getNodeIdProtoIterable(
+ final List nodeList) {
+ maybeInitBuilder();
+ return new Iterable() {
+ @Override
+ public synchronized Iterator iterator() {
+ return new Iterator() {
+
+ Iterator iter = nodeList.iterator();
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public YarnProtos.NodeIdProto next() {
+ return ProtoUtils.convertToProtoFormat(iter.next());
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java
new file mode 100644
index 0000000..e1a7726
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/DistSchedRegisterResponsePBImpl.java
@@ -0,0 +1,274 @@
+/**
+ * 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.protocolrecords.impl.pb;
+
+import com.google.protobuf.TextFormat;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.DistSchedRegisterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.DistSchedRegisterResponseProto;
+
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class DistSchedRegisterResponsePBImpl extends DistSchedRegisterResponse {
+
+ DistSchedRegisterResponseProto proto =
+ DistSchedRegisterResponseProto.getDefaultInstance();
+ DistSchedRegisterResponseProto.Builder builder = null;
+ boolean viaProto = false;
+
+ private Resource maxAllocatableCapability;
+ private Resource minAllocatableCapability;
+ private List nodesForScheduling;
+ private RegisterApplicationMasterResponse registerApplicationMasterResponse;
+
+ public DistSchedRegisterResponsePBImpl() {
+ builder = DistSchedRegisterResponseProto.newBuilder();
+ }
+
+ public DistSchedRegisterResponsePBImpl(DistSchedRegisterResponseProto proto) {
+ this.proto = proto;
+ viaProto = true;
+ }
+
+ public DistSchedRegisterResponseProto getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
+ private void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = DistSchedRegisterResponseProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
+ private synchronized void mergeLocalToProto() {
+ if (viaProto)
+ maybeInitBuilder();
+ mergeLocalToBuilder();
+ proto = builder.build();
+ viaProto = true;
+ }
+
+ private synchronized void mergeLocalToBuilder() {
+ if (this.nodesForScheduling != null) {
+ builder.clearNodesForScheduling();
+ Iterable iterable =
+ getNodeIdProtoIterable(this.nodesForScheduling);
+ builder.addAllNodesForScheduling(iterable);
+ }
+ if (this.maxAllocatableCapability != null) {
+ builder.setMaxAllocCapability(
+ ProtoUtils.convertToProtoFormat(this.maxAllocatableCapability));
+ }
+ if (this.minAllocatableCapability != null) {
+ builder.setMaxAllocCapability(
+ ProtoUtils.convertToProtoFormat(this.minAllocatableCapability));
+ }
+ if (this.registerApplicationMasterResponse != null) {
+ builder.setRegisterResponse(
+ ((RegisterApplicationMasterResponsePBImpl)
+ this.registerApplicationMasterResponse).getProto());
+ }
+ }
+
+ @Override
+ public void setRegisterResponse(RegisterApplicationMasterResponse resp) {
+ maybeInitBuilder();
+ if(registerApplicationMasterResponse == null) {
+ builder.clearRegisterResponse();
+ }
+ this.registerApplicationMasterResponse = resp;
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse getRegisterResponse() {
+ if (this.registerApplicationMasterResponse != null) {
+ return this.registerApplicationMasterResponse;
+ }
+
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasRegisterResponse()) {
+ return null;
+ }
+
+ this.registerApplicationMasterResponse =
+ new RegisterApplicationMasterResponsePBImpl(p.getRegisterResponse());
+ return this.registerApplicationMasterResponse;
+ }
+
+ @Override
+ public void setMaxAllocatableCapabilty(Resource maxResource) {
+ maybeInitBuilder();
+ if(maxAllocatableCapability == null) {
+ builder.clearMaxAllocCapability();
+ }
+ this.maxAllocatableCapability = maxResource;
+ }
+
+ @Override
+ public Resource getMaxAllocatableCapabilty() {
+ if (this.maxAllocatableCapability != null) {
+ return this.maxAllocatableCapability;
+ }
+
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasMaxAllocCapability()) {
+ return null;
+ }
+
+ this.maxAllocatableCapability =
+ ProtoUtils.convertFromProtoFormat(p.getMaxAllocCapability());
+ return this.maxAllocatableCapability;
+ }
+
+ @Override
+ public void setMinAllocatableCapabilty(Resource minResource) {
+ maybeInitBuilder();
+ if(minAllocatableCapability == null) {
+ builder.clearMinAllocCapability();
+ }
+ this.minAllocatableCapability = minResource;
+ }
+
+ @Override
+ public Resource getMinAllocatableCapabilty() {
+ if (this.minAllocatableCapability != null) {
+ return this.minAllocatableCapability;
+ }
+
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasMinAllocCapability()) {
+ return null;
+ }
+
+ this.minAllocatableCapability =
+ ProtoUtils.convertFromProtoFormat(p.getMinAllocCapability());
+ return this.minAllocatableCapability;
+ }
+
+ @Override
+ public void setContainerTokenExpiryInterval(int interval) {
+ maybeInitBuilder();
+ builder.setContainerTokenExpiryInterval(interval);
+ }
+
+ @Override
+ public int getContainerTokenExpiryInterval() {
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasContainerTokenExpiryInterval()) {
+ return 0;
+ }
+ return p.getContainerTokenExpiryInterval();
+ }
+
+ @Override
+ public void setContainerIdStart(long containerIdStart) {
+ maybeInitBuilder();
+ builder.setContainerIdStart(containerIdStart);
+ }
+
+ @Override
+ public long getContainerIdStart() {
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasContainerIdStart()) {
+ return 0;
+ }
+ return p.getContainerIdStart();
+ }
+
+
+ @Override
+ public void setNodesForScheduling(List nodesForScheduling) {
+ maybeInitBuilder();
+ if (nodesForScheduling == null || nodesForScheduling.isEmpty()) {
+ if (this.nodesForScheduling != null) {
+ this.nodesForScheduling.clear();
+ }
+ builder.clearNodesForScheduling();
+ return;
+ }
+ this.nodesForScheduling = new ArrayList<>();
+ this.nodesForScheduling.addAll(nodesForScheduling);
+ }
+
+ @Override
+ public List getNodesForScheduling() {
+ if (nodesForScheduling != null) {
+ return nodesForScheduling;
+ }
+ initLocalNodesForSchedulingList();
+ return nodesForScheduling;
+ }
+
+ private synchronized void initLocalNodesForSchedulingList() {
+ DistSchedRegisterResponseProtoOrBuilder p = viaProto ? proto : builder;
+ List list = p.getNodesForSchedulingList();
+ nodesForScheduling = new ArrayList<>();
+ if (list != null) {
+ for (YarnProtos.NodeIdProto t : list) {
+ nodesForScheduling.add(ProtoUtils.convertFromProtoFormat(t));
+ }
+ }
+ }
+ private synchronized Iterable getNodeIdProtoIterable(
+ final List nodeList) {
+ maybeInitBuilder();
+ return new Iterable() {
+ @Override
+ public synchronized Iterator iterator() {
+ return new Iterator() {
+
+ Iterator iter = nodeList.iterator();
+
+ @Override
+ public boolean hasNext() {
+ return iter.hasNext();
+ }
+
+ @Override
+ public YarnProtos.NodeIdProto next() {
+ return ProtoUtils.convertToProtoFormat(iter.next());
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public String toString() {
+ return TextFormat.shortDebugString(getProto());
+ }
+}
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..c90b928 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,15 +24,18 @@
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;
import com.google.protobuf.TextFormat;
+
@Private
@Unstable
public class ContainerStatusPBImpl extends ContainerStatus {
@@ -79,6 +82,7 @@ public String toString() {
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 +111,25 @@ private synchronized void maybeInitBuilder() {
}
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() {
@@ -205,6 +227,14 @@ private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
private ContainerIdProto convertToProtoFormat(ContainerId t) {
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();
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..3f95f10 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.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;
@@ -36,8 +37,10 @@
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.proto.YarnProtos;
import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
@@ -54,6 +57,7 @@
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 +286,25 @@ public static ContainerTypeProto convertToProtoFormat(ContainerType e) {
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());
+ }
+
+ /*
+ * Resource
+ */
+ public static synchronized YarnProtos.ResourceProto convertToProtoFormat(Resource r) {
+ return ((ResourcePBImpl) r).getProto();
+ }
+
+ public static Resource convertFromProtoFormat(YarnProtos.ResourceProto resource) {
+ return new ResourcePBImpl(resource);
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
index fc09cef..c6eaee9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java
@@ -21,9 +21,11 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.proto.YarnProtos;
import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
@@ -206,4 +208,24 @@ public void setNodeLabelExpression(String nodeLabelExpression) {
}
builder.setNodeLabelExpression(nodeLabelExpression);
}
+
+ @Override
+ public ExecutionType getExecutionType() {
+ ResourceRequestProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasExecutionType()) {
+ return null;
+ }
+ return ProtoUtils.convertFromProtoFormat(p.getExecutionType());
+ }
+
+ @Override
+ public void setExecutionType(ExecutionType execType) {
+ maybeInitBuilder();
+ if (execType == null) {
+ builder.clearExecutionType();
+ return;
+ }
+ builder.setExecutionType(ProtoUtils.convertToProtoFormat(execType));
+ }
+
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
index b29263e..d41cfa2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
@@ -36,6 +36,7 @@
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -95,7 +96,8 @@ protected InetSocketAddress getRMAddress(YarnConfiguration conf,
YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
- } else if (protocol == ApplicationMasterProtocol.class) {
+ } else if (protocol == ApplicationMasterProtocol.class
+ || protocol == DistributedSchedulerProtocol.class ) {
setAMRMTokenService(conf);
return conf.getSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
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..ad6e70a 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.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.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;
@@ -68,7 +70,7 @@ public ContainerTokenIdentifier(ContainerId containerID,
int masterKeyId, long rmIdentifier, Priority priority, long creationTime) {
this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
rmIdentifier, priority, creationTime, null,
- CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
+ CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK, ExecutionType.GUARANTEED);
}
public ContainerTokenIdentifier(ContainerId containerID, String hostName,
@@ -77,14 +79,14 @@ public ContainerTokenIdentifier(ContainerId containerID, String hostName,
LogAggregationContext logAggregationContext, String nodeLabelExpression) {
this(containerID, hostName, appSubmitter, r, expiryTimeStamp, masterKeyId,
rmIdentifier, priority, creationTime, logAggregationContext,
- nodeLabelExpression, ContainerType.TASK);
+ nodeLabelExpression, ContainerType.TASK, 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) {
+ ContainerType containerType, ExecutionType executionType) {
ContainerTokenIdentifierProto.Builder builder =
ContainerTokenIdentifierProto.newBuilder();
if (containerID != null) {
@@ -112,6 +114,7 @@ public ContainerTokenIdentifier(ContainerId containerID, String hostName,
builder.setNodeLabelExpression(nodeLabelExpression);
}
builder.setContainerType(convertToProtoFormat(containerType));
+ builder.setExecutionType(convertToProtoFormat(executionType));
proto = builder.build();
}
@@ -180,6 +183,17 @@ public ContainerType getContainerType(){
}
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 +279,13 @@ private ContainerType convertFromProtoFormat(
ContainerTypeProto containerType) {
return ProtoUtils.convertFromProtoFormat(containerType);
}
+
+ private ExecutionTypeProto convertToProtoFormat(ExecutionType executionType) {
+ return ProtoUtils.convertToProtoFormat(executionType);
+ }
+
+ private ExecutionType convertFromProtoFormat(
+ ExecutionTypeProto executionType) {
+ return ProtoUtils.convertFromProtoFormat(executionType);
+ }
}
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..2403fc1 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;
}
message ClientToAMTokenIdentifierProto {
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..2611603 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.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 void testContainerTokenIdentifier() throws IOException {
Assert.assertEquals(ContainerType.TASK,
anotherToken.getContainerType());
+
+ Assert.assertEquals(ExecutionType.GUARANTEED,
+ anotherToken.getExecutionType());
}
@Test
@@ -369,10 +373,10 @@ public void testAMContainerTokenIdentifier() throws IOException {
Priority priority = Priority.newInstance(1);
long creationTime = 1000;
- ContainerTokenIdentifier token =
- new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
- expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
- null, CommonNodeLabelsManager.NO_LABEL, ContainerType.APPLICATION_MASTER);
+ ContainerTokenIdentifier token = new ContainerTokenIdentifier(containerID,
+ hostName, appSubmitter, r, expiryTimeStamp, masterKeyId, rmIdentifier,
+ priority, creationTime, null, CommonNodeLabelsManager.NO_LABEL,
+ ContainerType.APPLICATION_MASTER, ExecutionType.GUARANTEED);
ContainerTokenIdentifier anotherToken = new ContainerTokenIdentifier();
@@ -383,11 +387,14 @@ public void testAMContainerTokenIdentifier() throws IOException {
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);
+ token = new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
+ expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
+ null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
+ ExecutionType.QUEUEABLE);
anotherToken = new ContainerTokenIdentifier();
@@ -398,6 +405,9 @@ public void testAMContainerTokenIdentifier() throws IOException {
Assert.assertEquals(ContainerType.TASK,
anotherToken.getContainerType());
+
+ Assert.assertEquals(ExecutionType.QUEUEABLE,
+ anotherToken.getExecutionType());
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 9c2d1fb..baae949 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -32,6 +32,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.QueueableContainerAllocator;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -87,4 +88,8 @@
ConcurrentLinkedQueue
getLogAggregationStatusForApps();
+
+ boolean isDistributedSchedulingEnabled();
+
+ QueueableContainerAllocator getContainerAllocator();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 04e383f..ff432d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -69,6 +69,7 @@
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMLeveldbStateStoreService;
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.QueueableContainerAllocator;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer;
@@ -187,9 +188,9 @@ protected DeletionService createDeletionService(ContainerExecutor exec) {
protected NMContext createNMContext(
NMContainerTokenSecretManager containerTokenSecretManager,
NMTokenSecretManagerInNM nmTokenSecretManager,
- NMStateStoreService stateStore) {
+ NMStateStoreService stateStore, boolean isDistSchedulerEnabled) {
return new NMContext(containerTokenSecretManager, nmTokenSecretManager,
- dirsHandler, aclsManager, stateStore);
+ dirsHandler, aclsManager, stateStore, isDistSchedulerEnabled);
}
protected void doSecureLogin() throws IOException {
@@ -310,8 +311,12 @@ protected void serviceInit(Configuration conf) throws Exception {
getNodeHealthScriptRunner(conf), dirsHandler);
addService(nodeHealthChecker);
+ boolean isDistSchedulingEnabled =
+ conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
+ YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT);
+
this.context = createNMContext(containerTokenSecretManager,
- nmTokenSecretManager, nmStore);
+ nmTokenSecretManager, nmStore, isDistSchedulingEnabled);
nodeLabelsProvider = createNodeLabelsProvider(conf);
@@ -340,6 +345,10 @@ protected void serviceInit(Configuration conf) throws Exception {
addService(webServer);
((NMContext) context).setWebServer(webServer);
+ ((NMContext) context).setQueueableContainerAllocator(
+ new QueueableContainerAllocator(nodeStatusUpdater, context,
+ webServer.getPort()));
+
dispatcher.register(ContainerManagerEventType.class, containerManager);
dispatcher.register(NodeManagerEventType.class, this);
addService(dispatcher);
@@ -461,11 +470,14 @@ public void run() {
private boolean isDecommissioned = false;
private final ConcurrentLinkedQueue
logAggregationReportForApps;
+ private final boolean isDistSchedulingEnabled;
+
+ private QueueableContainerAllocator containerAllocator;
public NMContext(NMContainerTokenSecretManager containerTokenSecretManager,
NMTokenSecretManagerInNM nmTokenSecretManager,
LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
- NMStateStoreService stateStore) {
+ NMStateStoreService stateStore, boolean isDistSchedulingEnabled) {
this.containerTokenSecretManager = containerTokenSecretManager;
this.nmTokenSecretManager = nmTokenSecretManager;
this.dirsHandler = dirsHandler;
@@ -476,6 +488,7 @@ public NMContext(NMContainerTokenSecretManager containerTokenSecretManager,
this.stateStore = stateStore;
this.logAggregationReportForApps = new ConcurrentLinkedQueue<
LogAggregationReport>();
+ this.isDistSchedulingEnabled = isDistSchedulingEnabled;
}
/**
@@ -588,6 +601,21 @@ public void setSystemCrendentialsForApps(
getLogAggregationStatusForApps() {
return this.logAggregationReportForApps;
}
+
+ @Override
+ public boolean isDistributedSchedulingEnabled() {
+ return isDistSchedulingEnabled;
+ }
+
+ public void setQueueableContainerAllocator(
+ QueueableContainerAllocator containerAllocator) {
+ this.containerAllocator = containerAllocator;
+ }
+
+ @Override
+ public QueueableContainerAllocator getContainerAllocator() {
+ return containerAllocator;
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index bd6538c..c564409 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -63,6 +63,8 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
+
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.LocalScheduler;
import org.apache.hadoop.yarn.server.security.MasterKeyData;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
@@ -464,6 +466,13 @@ protected RequestInterceptor createRequestInterceptorChain() {
interceptorClassNames.add(item.trim());
}
+ // Make sure LocalScheduler is present at the end
+ // of the chain.. and remove the DefaultRequestInterceptor
+ if (this.nmContext.isDistributedSchedulingEnabled()) {
+ interceptorClassNames.remove(DefaultRequestInterceptor.class.getName());
+ interceptorClassNames.add(LocalScheduler.class.getName());
+ }
+
return interceptorClassNames;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index cd5ed88..3edd1da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -226,7 +226,8 @@ public void handle(AuxServicesEvent event) {
serv.initializeContainer(new ContainerInitializationContext(
event.getUser(), event.getContainer().getContainerId(),
event.getContainer().getResource(), event.getContainer()
- .getContainerTokenIdentifier().getContainerType()));
+ .getContainerTokenIdentifier().getContainerType(), event.getContainer()
+ .getContainerTokenIdentifier().getExecutionType()));
} catch (Throwable th) {
logWarningWhenAuxServiceThrowExceptions(serv,
AuxServicesEventType.CONTAINER_INIT, th);
@@ -239,7 +240,8 @@ public void handle(AuxServicesEvent event) {
serv.stopContainer(new ContainerTerminationContext(
event.getUser(), event.getContainer().getContainerId(),
event.getContainer().getResource(), event.getContainer()
- .getContainerTokenIdentifier().getContainerType()));
+ .getContainerTokenIdentifier().getContainerType(), event.getContainer()
+ .getContainerTokenIdentifier().getExecutionType()));
} catch (Throwable th) {
logWarningWhenAuxServiceThrowExceptions(serv,
AuxServicesEventType.CONTAINER_STOP, th);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
new file mode 100644
index 0000000..e82c7b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/LocalScheduler.java
@@ -0,0 +1,393 @@
+/**
+ * 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.server.nodemanager.scheduler;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy
+ .AMRMProxyApplicationContext;
+import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AbstractRequestInterceptor;
+
+
+
+import org.apache.hadoop.yarn.server.nodemanager.security
+ .NMTokenSecretManagerInNM;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+public final class LocalScheduler extends AbstractRequestInterceptor {
+
+ class PartitionedResourceRequests {
+ private List guaranteed = new ArrayList<>();
+ private List queueable = new ArrayList<>();
+ public List getGuaranteed() {
+ return guaranteed;
+ }
+ public List getQueueable() {
+ return queueable;
+ }
+ }
+
+ public static class PriorityComparator implements Comparator {
+ @Override
+ public int compare(Priority o1, Priority o2) {
+ return o1.getPriority() - o2.getPriority();
+ }
+ }
+
+ class DistSchedulerParams {
+ int maxAllocMb;
+ int minAllocMb;
+ int maxVCores;
+ int minVCores;
+ int containerTokenExpiryInterval;
+ }
+
+ private static final Logger LOG = LoggerFactory
+ .getLogger(LocalScheduler.class);
+
+ private int numAllocatedContainers;
+ private DistSchedulerParams appParams = new DistSchedulerParams();
+ private final QueueableContainerAllocator.ContainerIdCounter containerIdCounter =
+ new QueueableContainerAllocator.ContainerIdCounter();
+ private Map nodeList = new HashMap<>();
+ private Map nodeTokens = new HashMap<>();
+ final Set blacklist = new TreeSet<>();
+ final Set priorities = new TreeSet<>(new PriorityComparator());
+ final Map>> requests =
+ new HashMap<>();
+
+ private DistributedSchedulerProtocol rmClient;
+ private UserGroupInformation user = null;
+
+ private ApplicationAttemptId applicationAttemptId;
+ private QueueableContainerAllocator containerAllocator;
+ private NMTokenSecretManagerInNM nmSecretManager;
+ private String appSubmitter;
+
+ public void init(AMRMProxyApplicationContext appContext) {
+ super.init(appContext);
+ initLocal(appContext.getApplicationAttemptId(),
+ appContext.getNMCotext().getContainerAllocator(),
+ appContext.getNMCotext().getNMTokenSecretManager(),
+ appContext.getUser());
+ try {
+ user =
+ UserGroupInformation.createProxyUser(applicationAttemptId.toString(),
+ UserGroupInformation.getCurrentUser());
+ user.addToken(appContext.getAMRMToken());
+ final Configuration conf = this.getConf();
+
+ rmClient =
+ user.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public DistributedSchedulerProtocol run() throws Exception {
+ return ClientRMProxy.createRMProxy(conf,
+ DistributedSchedulerProtocol.class);
+ }
+ });
+ } catch (IOException e) {
+ String message =
+ "Error while creating of RM app master service proxy for attemptId:"
+ + applicationAttemptId.toString();
+ if (user != null) {
+ message += ", user: " + user;
+ }
+
+ LOG.info(message);
+ throw new YarnRuntimeException(message, e);
+ } catch (Exception e) {
+ throw new YarnRuntimeException(e);
+ }
+ }
+
+ @VisibleForTesting
+ void initLocal(ApplicationAttemptId applicationAttemptId,
+ QueueableContainerAllocator containerAllocator,
+ NMTokenSecretManagerInNM nmSecretManager, String appSubmitter) {
+ this.applicationAttemptId = applicationAttemptId;
+ this.containerAllocator = containerAllocator;
+ this.nmSecretManager = nmSecretManager;
+ this.appSubmitter = appSubmitter;
+ }
+
+ @VisibleForTesting
+ void setRmClient(DistributedSchedulerProtocol client) {
+ this.rmClient = client;
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ LOG.info("Forwarding registration request to the" +
+ "Distributed Scheduler Service on YARN RM");
+ DistSchedRegisterResponse dsResp =
+ rmClient.registerApplicationMasterForDistributedScheduling(request);
+ updateParameters(dsResp);
+ return dsResp.getRegisterResponse();
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return getNextInterceptor().finishApplicationMaster(request);
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ PartitionedResourceRequests partitionedAsks = partitionAskList(request
+ .getAskList());
+ request.setAskList(partitionedAsks.getGuaranteed());
+ List releasedContainers = request.getReleaseList();
+ int numReleasedContainers = releasedContainers.size();
+ if (numReleasedContainers > 0) {
+ LOG.info("AttemptID: " + applicationAttemptId + " released: "
+ + numReleasedContainers);
+ numAllocatedContainers -= numReleasedContainers;
+ }
+
+ // Also, update black list
+ ResourceBlacklistRequest rbr = request.getResourceBlacklistRequest();
+ if (rbr != null) {
+ blacklist.removeAll(rbr.getBlacklistRemovals());
+ blacklist.addAll(rbr.getBlacklistAdditions());
+ }
+
+ updateResourceAsk(partitionedAsks.getQueueable());
+ List allocatedContainers = new ArrayList<>();
+ for (Priority priority : priorities) {
+ for (Map reqMap :
+ requests.get(priority).values()) {
+ Map> allocated =
+ containerAllocator.allocate(this.appParams, containerIdCounter,
+ reqMap.values(), blacklist, applicationAttemptId, nodeList,
+ appSubmitter);
+ for (Map.Entry> e : allocated.entrySet()) {
+ updateAllocation(e.getKey(), e.getValue());
+ allocatedContainers.addAll(e.getValue());
+ }
+ }
+ }
+
+ DistSchedAllocateResponse dsResp =
+ rmClient.allocateForDistributedScheduling(request);
+
+ setNodeList(dsResp.getNodesForScheduling());
+ List nmTokens = dsResp.getAllocateResponse().getNMTokens();
+ for (NMToken nmToken : nmTokens) {
+ nodeTokens.put(nmToken.getNodeId(), nmToken);
+ }
+ int numCompletedContainers = 0;
+ List completedContainers =
+ dsResp.getAllocateResponse().getCompletedContainersStatuses();
+
+ // Only account for queueable containers
+ for (ContainerStatus cs : completedContainers) {
+ if (cs.getExecutionType() == ExecutionType.QUEUEABLE)
+ numCompletedContainers++;
+ }
+ numAllocatedContainers -= numCompletedContainers;
+
+ updateResponseWithNMTokens(
+ dsResp.getAllocateResponse(), nmTokens, allocatedContainers);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Number of queueable containers currently allocated by application: "
+ + numAllocatedContainers);
+ }
+ return dsResp.getAllocateResponse();
+ }
+
+ private void updateResponseWithNMTokens(AllocateResponse response,
+ List nmTokens, List allocatedContainers) {
+ List newTokens = new ArrayList<>();
+ if (allocatedContainers.size() > 0) {
+ response.getAllocatedContainers().addAll(allocatedContainers);
+ for (Container alloc : allocatedContainers) {
+ if (!nodeTokens.containsKey(alloc.getNodeId())) {
+ newTokens.add(nmSecretManager.generateNMToken(appSubmitter, alloc));
+ }
+ }
+ List retTokens = new ArrayList<>(nmTokens);
+ retTokens.addAll(newTokens);
+ response.setNMTokens(retTokens);
+ }
+ }
+
+ private PartitionedResourceRequests partitionAskList(List
+ askList) {
+ PartitionedResourceRequests partitionedRequests =
+ new PartitionedResourceRequests();
+ for (ResourceRequest rr : askList) {
+ if (rr.getExecutionType() == ExecutionType.QUEUEABLE) {
+ partitionedRequests.getQueueable().add(rr);
+ } else {
+ partitionedRequests.getGuaranteed().add(rr);
+ }
+ }
+ return partitionedRequests;
+ }
+
+ private void updateParameters(
+ DistSchedRegisterResponse registerResponse) {
+ appParams.minAllocMb = registerResponse.getMinAllocatableCapabilty()
+ .getMemory();
+ appParams.maxAllocMb = registerResponse.getMaxAllocatableCapabilty()
+ .getMemory();
+ appParams.minVCores = registerResponse.getMinAllocatableCapabilty()
+ .getVirtualCores();
+ appParams.maxVCores = registerResponse.getMaxAllocatableCapabilty()
+ .getVirtualCores();
+ appParams.containerTokenExpiryInterval = registerResponse
+ .getContainerTokenExpiryInterval();
+
+ containerIdCounter
+ .resetContainerIdCounter(registerResponse.getContainerIdStart());
+ setNodeList(registerResponse.getNodesForScheduling());
+ }
+
+ public void updateResourceAsk(List requests) {
+ // Update resource requests
+ for (ResourceRequest request : requests) {
+ // Handling locality for queueable tokens is optional; we rely on
+ // "anyAsk" to drive allocations
+ Priority priority = request.getPriority();
+ String resourceName = request.getResourceName();
+
+ if (!ResourceRequest.isAnyLocation(request.getResourceName())) {
+ continue;
+ }
+
+ if (request.getNumContainers() == 0) {
+ continue;
+ }
+
+ Map> asks =
+ this.requests.get(priority);
+ if (asks == null) {
+ asks = new HashMap<>();
+ this.requests.put(priority, asks);
+ this.priorities.add(priority);
+ }
+
+ Map reqMap = asks.get(resourceName);
+ if (reqMap == null) {
+ reqMap = new HashMap<>();
+ asks.put(resourceName, reqMap);
+ }
+
+ ResourceRequest resourceRequest = reqMap.get(request.getCapability());
+ if (resourceRequest == null) {
+ resourceRequest = request;
+ reqMap.put(request.getCapability(), request);
+ } else {
+ resourceRequest.setNumContainers(
+ resourceRequest.getNumContainers() + request.getNumContainers());
+ }
+ if (ResourceRequest.isAnyLocation(request.getResourceName())) {
+ LOG.info("# of requests in ANY (at priority = " + priority + ", "
+ + "with capability = " + request.getCapability() + " ) : "
+ + resourceRequest.getNumContainers());
+ }
+ }
+ }
+
+ public void updateAllocation(Resource capability,
+ List allocatedContainers) {
+ numAllocatedContainers += allocatedContainers.size();
+ for (Container c : allocatedContainers) {
+ Map> asks = this
+ .requests.get(c.getPriority());
+
+ if (asks == null)
+ continue;
+
+ // Host specific accounting
+ removeFromReqMap(capability, asks.get(c.getNodeId().getHost()));
+
+ // any ask accounting
+ removeFromReqMap(capability, asks.get(ResourceRequest.ANY));
+ }
+ }
+
+ private void removeFromReqMap(Resource capability, Map rrMap) {
+ if (rrMap != null) {
+ ResourceRequest rr = rrMap.get(capability);
+ if (rr != null) {
+ rr.setNumContainers(rr.getNumContainers() - 1);
+ if (rr.getNumContainers() == 0)
+ rrMap.remove(capability);
+ }
+ }
+ }
+
+ private void setNodeList(List nodeList) {
+ this.nodeList.clear();
+ addToNodeList(nodeList);
+ }
+
+ private void addToNodeList(List nodes) {
+ for (NodeId n : nodes) {
+ this.nodeList.put(n.getHost(), n);
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/QueueableContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/QueueableContainerAllocator.java
new file mode 100644
index 0000000..31a1c15
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/QueueableContainerAllocator.java
@@ -0,0 +1,183 @@
+/**
+ * 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.server.nodemanager.scheduler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler.LocalScheduler.DistSchedulerParams;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+
+import java.net.InetSocketAddress;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QueueableContainerAllocator {
+
+ private static final Log LOG =
+ LogFactory.getLog(QueueableContainerAllocator.class);
+
+ static class ContainerIdCounter {
+ final AtomicLong containerIdCounter = new AtomicLong(1);
+
+ void resetContainerIdCounter(long containerIdStart) {
+ this.containerIdCounter.set(containerIdStart);
+ }
+
+ long generateContainerId() {
+ return this.containerIdCounter.decrementAndGet();
+ }
+ }
+
+ private final NodeStatusUpdater nodeStatusUpdater;
+ private final Context context;
+ private int webpagePort;
+
+ public QueueableContainerAllocator(NodeStatusUpdater nodeStatusUpdater,
+ Context context, int webpagePort) {
+ this.nodeStatusUpdater = nodeStatusUpdater;
+ this.context = context;
+ this.webpagePort = webpagePort;
+ }
+
+ public Map> allocate(DistSchedulerParams appParams,
+ ContainerIdCounter idCounter, Collection resourceAsks,
+ Set blacklist, ApplicationAttemptId appAttId,
+ Map allNodes, String userName) throws YarnException {
+ Map> containers = new HashMap<>();
+ Set nodesAllocated = new HashSet<>();
+ List anyAsks = new ArrayList<>(resourceAsks);
+ for (ResourceRequest anyAsk : anyAsks) {
+ allocateQueueableContainers(appParams, idCounter, blacklist, appAttId,
+ allNodes, userName, containers, nodesAllocated, anyAsk);
+ }
+ if (resourceAsks.size() > 0) {
+ LOG.info("Queueable allocation requested for: " + resourceAsks.size()
+ + " containers; allocated = " + containers.size());
+ }
+ return containers;
+ }
+
+ private void allocateQueueableContainers(DistSchedulerParams appParams,
+ ContainerIdCounter idCounter, Set blacklist,
+ ApplicationAttemptId id, Map allNodes, String userName,
+ Map> containers, Set nodesAllocated,
+ ResourceRequest anyAsk) throws YarnException {
+ int toAllocate = anyAsk.getNumContainers()
+ - (containers.isEmpty() ?
+ 0 : containers.get(anyAsk.getCapability()).size());
+
+ List topKNodesLeft = new ArrayList<>();
+ for (String s : allNodes.keySet()) {
+ // Bias away from whatever we have already allocated and respect blacklist
+ if (nodesAllocated.contains(s) || blacklist.contains(s))
+ continue;
+ topKNodesLeft.add(s);
+ }
+ int numAllocated = 0;
+ int nextNodeToAllocate = 0;
+ for (int numCont = 0; numCont < toAllocate; numCont++) {
+ String topNode = topKNodesLeft.get(nextNodeToAllocate);
+ nextNodeToAllocate++;
+ nextNodeToAllocate %= topKNodesLeft.size();
+ NodeId nodeId = allNodes.get(topNode);
+ Container container = buildContainer(appParams, idCounter, anyAsk, id,
+ userName, nodeId);
+ List cList = containers.get(anyAsk.getCapability());
+ if (cList == null) {
+ cList = new ArrayList<>();
+ containers.put(anyAsk.getCapability(), cList);
+ }
+ cList.add(container);
+ numAllocated++;
+ LOG.info("Allocated " + numAllocated + " queueable containers.");
+ }
+ }
+
+ private Container buildContainer(DistSchedulerParams appParams,
+ ContainerIdCounter idCounter, ResourceRequest rr, ApplicationAttemptId id,
+ String userName, NodeId nodeId) throws YarnException {
+ ContainerId cId =
+ ContainerId.newContainerId(id, idCounter.generateContainerId());
+ Resource capability = fixCapability(appParams, rr.getCapability());
+ long currTime = System.currentTimeMillis();
+ ContainerTokenIdentifier containerTokenIdentifier =
+ new ContainerTokenIdentifier(
+ cId, nodeId.getHost(), userName, capability,
+ currTime + appParams.containerTokenExpiryInterval,
+ context.getContainerTokenSecretManager().getCurrentKey().getKeyId(),
+ nodeStatusUpdater.getRMIdentifier(), rr.getPriority(), currTime,
+ null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
+ ExecutionType.QUEUEABLE);
+ byte[] pwd =
+ context.getContainerTokenSecretManager().createPassword(
+ containerTokenIdentifier);
+ Token containerToken = newContainerToken(nodeId, pwd,
+ containerTokenIdentifier);
+ Container container = BuilderUtils.newContainer(
+ cId, nodeId, nodeId.getHost() + ":" + webpagePort,
+ capability, rr.getPriority(), containerToken);
+ return container;
+ }
+
+ private Resource fixCapability(DistSchedulerParams appParams,
+ Resource capability) {
+ return Resource.newInstance(
+ fixMemory(appParams, capability.getMemory()),
+ fixVCores(appParams, capability.getVirtualCores()));
+ }
+
+ // #!# Round to the next multiple of container dimension
+ private int fixMemory(DistSchedulerParams appParams, int memory) {
+ if (memory > appParams.maxAllocMb) {
+ return appParams.maxAllocMb;
+ }
+ return appParams.minAllocMb *
+ ((int) Math.ceil((float) memory / appParams.minAllocMb));
+ }
+
+ // #!# Round to the select the right number of virtual core
+ private int fixVCores(DistSchedulerParams appParams, int vCores) {
+ if (vCores > appParams.maxVCores) {
+ return appParams.maxVCores;
+ }
+ return vCores < appParams.minVCores ? appParams.minVCores : vCores;
+ }
+
+ public static Token newContainerToken(NodeId nodeId, byte[] password,
+ ContainerTokenIdentifier tokenIdentifier) {
+ // RPC layer client expects ip:port as service for tokens
+ InetSocketAddress addr = NetUtils.createSocketAddrForHost(nodeId.getHost(),
+ nodeId.getPort());
+ // NOTE: use SecurityUtil.setTokenService if this becomes a "real" token
+ Token containerToken = Token.newInstance(tokenIdentifier.getBytes(),
+ ContainerTokenIdentifier.KIND.toString(), password, SecurityUtil
+ .buildTokenService(addr).toString());
+ return containerToken;
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
index f6169e7..86cce35 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/security/NMTokenSecretManagerInNM.java
@@ -29,7 +29,10 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
@@ -50,7 +53,7 @@
private final Map oldMasterKeys;
private final Map> appToAppAttemptMap;
private final NMStateStoreService stateStore;
- private NodeId nodeId;
+ private NodeId nodeId;
public NMTokenSecretManagerInNM() {
this(new NMNullStateStoreService());
@@ -276,4 +279,23 @@ private void removeAppAttemptKey(ApplicationAttemptId attempt) {
LOG.error("Unable to remove master key for application " + attempt, e);
}
}
+
+ /**
+ * Used by the Distributed Scheduler framework to generate NMTokens
+ * @param applicationSubmitter
+ * @param container
+ * @return NMToken
+ */
+ public NMToken generateNMToken(
+ String applicationSubmitter, Container container) {
+ this.readLock.lock();
+ try {
+ Token token =
+ createNMToken(container.getId().getApplicationAttemptId(),
+ container.getNodeId(), applicationSubmitter);
+ return NMToken.newInstance(container.getNodeId(), token);
+ } finally {
+ this.readLock.unlock();
+ }
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
index 3dc62bc..6885804 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
@@ -80,7 +80,7 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
Context context = new NMContext(new NMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInNM(), null, null,
- new NMNullStateStoreService()) {
+ new NMNullStateStoreService(), false) {
@Override
public int getHttpPort() {
return 1234;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index 90804b8..d0d437c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -1565,7 +1565,7 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
protected NMContext createNMContext(
NMContainerTokenSecretManager containerTokenSecretManager,
NMTokenSecretManagerInNM nmTokenSecretManager,
- NMStateStoreService store) {
+ NMStateStoreService store, boolean isDistributedSchedulingEnabled) {
return new MyNMContext(containerTokenSecretManager,
nmTokenSecretManager);
}
@@ -1800,7 +1800,7 @@ public MyNMContext(
NMContainerTokenSecretManager containerTokenSecretManager,
NMTokenSecretManagerInNM nmTokenSecretManager) {
super(containerTokenSecretManager, nmTokenSecretManager, null, null,
- new NMNullStateStoreService());
+ new NMNullStateStoreService(), false);
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index 9bc23f6..823860b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -66,6 +66,8 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.scheduler
+ .QueueableContainerAllocator;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -678,5 +680,14 @@ public NodeResourceMonitor getNodeResourceMonitor() {
return null;
}
+ @Override
+ public boolean isDistributedSchedulingEnabled() {
+ return false;
+ }
+
+ @Override
+ public QueueableContainerAllocator getContainerAllocator() {
+ return null;
+ }
}
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index c902fd5..d070bbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -110,7 +110,7 @@ public BaseContainerManagerTest() throws UnsupportedFileSystemException {
protected Configuration conf = new YarnConfiguration();
protected Context context = new NMContext(new NMContainerTokenSecretManager(
conf), new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), new NMNullStateStoreService()) {
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false) {
public int getHttpPort() {
return HTTP_PORT;
};
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 2e014de..dfb7a1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -471,7 +471,7 @@ private NMContext createContext(Configuration conf,
NMStateStoreService stateStore) {
NMContext context = new NMContext(new NMContainerTokenSecretManager(
conf), new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), stateStore){
+ new ApplicationACLsManager(conf), stateStore, false){
public int getHttpPort() {
return HTTP_PORT;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
index 0abae2b..6f3adb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
@@ -109,7 +109,7 @@
protected Context distContext = new NMContext(new NMContainerTokenSecretManager(
conf), new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), new NMNullStateStoreService()) {
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false) {
public int getHttpPort() {
return HTTP_PORT;
};
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java
index 9e08b7f..c768df1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalCacheDirectoryManager.java
@@ -81,7 +81,8 @@ public void testMinimumPerDirectoryFileLimit() {
NMContext nmContext =
new NMContext(new NMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), new NMNullStateStoreService());
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(),
+ false);
ResourceLocalizationService service =
new ResourceLocalizationService(null, null, null, null, nmContext);
try {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index 64d3d68..4c594e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -186,7 +186,7 @@ public void setup() throws IOException {
conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
nmContext = new NMContext(new NMContainerTokenSecretManager(
conf), new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), new NMNullStateStoreService());
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false);
}
@After
@@ -2365,7 +2365,7 @@ private ResourceLocalizationService createSpyService(
NMContext nmContext =
new NMContext(new NMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInNM(), null,
- new ApplicationACLsManager(conf), stateStore);
+ new ApplicationACLsManager(conf), stateStore, false);
ResourceLocalizationService rawService =
new ResourceLocalizationService(dispatcher, exec, delService,
dirsHandler, nmContext);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 101fef0..648cb62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -81,6 +81,7 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -1839,7 +1840,8 @@ private ContainerId createContainer(ApplicationAttemptId appAttemptId1,
ContainerTokenIdentifier containerToken = new ContainerTokenIdentifier(
containerId, context.getNodeId().toString(), user, r,
System.currentTimeMillis() + 100000L, 123, DUMMY_RM_IDENTIFIER,
- Priority.newInstance(0), 0, null, null, containerType);
+ Priority.newInstance(0), 0, null, null, containerType,
+ ExecutionType.GUARANTEED);
Container container = mock(Container.class);
context.getContainers().put(containerId, container);
when(container.getContainerTokenIdentifier()).thenReturn(containerToken);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
new file mode 100644
index 0000000..b003a99
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/TestLocalScheduler.java
@@ -0,0 +1,212 @@
+/**
+ * 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.server.nodemanager.scheduler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords
+ .RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+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;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
+import org.apache.hadoop.yarn.server.nodemanager.security
+ .NMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.nodemanager.security
+ .NMTokenSecretManagerInNM;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestLocalScheduler {
+
+ @Test
+ public void testLocalScheduler() throws Exception {
+
+ Configuration conf = new Configuration();
+ LocalScheduler localScheduler = new LocalScheduler();
+
+ NodeStatusUpdater nodeStatusUpdater = Mockito.mock(NodeStatusUpdater.class);
+ Mockito.when(nodeStatusUpdater.getRMIdentifier()).thenReturn(12345l);
+ Context context = Mockito.mock(Context.class);
+ NMContainerTokenSecretManager nmContainerTokenSecretManager = new
+ NMContainerTokenSecretManager(conf);
+ MasterKey mKey = new MasterKey() {
+ @Override
+ public int getKeyId() {
+ return 1;
+ }
+ @Override
+ public void setKeyId(int keyId) {}
+ @Override
+ public ByteBuffer getBytes() {
+ return ByteBuffer.allocate(8);
+ }
+ @Override
+ public void setBytes(ByteBuffer bytes) {}
+ };
+ nmContainerTokenSecretManager.setMasterKey(mKey);
+ Mockito.when(context.getContainerTokenSecretManager()).thenReturn
+ (nmContainerTokenSecretManager);
+ QueueableContainerAllocator containerAllocator =
+ new QueueableContainerAllocator(nodeStatusUpdater, context, 7777);
+
+ NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
+ new NMTokenSecretManagerInNM();
+ nmTokenSecretManagerInNM.setMasterKey(mKey);
+ localScheduler.initLocal(
+ ApplicationAttemptId.newInstance(ApplicationId.newInstance(1, 1), 1),
+ containerAllocator, nmTokenSecretManagerInNM, "test");
+
+ DistributedSchedulerProtocol protocol = Mockito.mock
+ (DistributedSchedulerProtocol.class);
+ localScheduler.setRmClient(protocol);
+
+ DistSchedRegisterResponse distSchedRegisterResponse =
+ Records.newRecord(DistSchedRegisterResponse.class);
+ distSchedRegisterResponse.setRegisterResponse(
+ Records.newRecord(RegisterApplicationMasterResponse.class));
+ distSchedRegisterResponse.setContainerTokenExpiryInterval(12345);
+ distSchedRegisterResponse.setContainerIdStart(0);
+ distSchedRegisterResponse.setMaxAllocatableCapabilty(
+ Resource.newInstance(1024, 4));
+ distSchedRegisterResponse.setMinAllocatableCapabilty(
+ Resource.newInstance(512, 2));
+ distSchedRegisterResponse.setNodesForScheduling(Arrays.asList(
+ NodeId.newInstance("a", 1), NodeId.newInstance("b", 2)));
+ Mockito.when(
+ protocol.registerApplicationMasterForDistributedScheduling(
+ Mockito.any(RegisterApplicationMasterRequest.class)))
+ .thenReturn(distSchedRegisterResponse);
+
+ localScheduler.registerApplicationMaster(
+ Records.newRecord(RegisterApplicationMasterRequest.class));
+
+ Mockito.when(
+ protocol.allocateForDistributedScheduling(
+ Mockito.any(AllocateRequest.class)))
+ .thenAnswer(new Answer() {
+ @Override
+ public DistSchedAllocateResponse answer(InvocationOnMock
+ invocationOnMock) throws Throwable {
+ return createAllocateResponse(Arrays.asList(
+ NodeId.newInstance("c", 3), NodeId.newInstance("d", 4)));
+ }
+ });
+
+ AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
+ ResourceRequest guaranteedReq = Records.newRecord(ResourceRequest.class);
+ guaranteedReq.setExecutionType(ExecutionType.GUARANTEED);
+ guaranteedReq.setNumContainers(5);
+ guaranteedReq.setCapability(Resource.newInstance(2048, 2));
+ guaranteedReq.setRelaxLocality(true);
+ guaranteedReq.setResourceName("*");
+ ResourceRequest queueableReq = Records.newRecord(ResourceRequest.class);
+ queueableReq.setExecutionType(ExecutionType.QUEUEABLE);
+ queueableReq.setNumContainers(4);
+ queueableReq.setCapability(Resource.newInstance(1024, 4));
+ queueableReq.setPriority(Priority.newInstance(100));
+ queueableReq.setRelaxLocality(true);
+ queueableReq.setResourceName("*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, queueableReq));
+
+ // Verify 4 containers were allocated
+ AllocateResponse allocateResponse = localScheduler.allocate(allocateRequest);
+ Assert.assertEquals(4, allocateResponse.getAllocatedContainers().size());
+
+ // Verify equal distribution on hosts a and b
+ // And None on c and d
+ Map> allocs = mapAllocs(allocateResponse);
+ Assert.assertEquals(2, allocs.get(NodeId.newInstance("a", 1)).size());
+ Assert.assertEquals(2, allocs.get(NodeId.newInstance("b", 2)).size());
+ Assert.assertNull(allocs.get(NodeId.newInstance("c", 3)));
+ Assert.assertNull(allocs.get(NodeId.newInstance("d", 4)));
+
+ // New Allocate request
+ allocateRequest = Records.newRecord(AllocateRequest.class);
+ queueableReq = Records.newRecord(ResourceRequest.class);
+ queueableReq.setExecutionType(ExecutionType.QUEUEABLE);
+ queueableReq.setNumContainers(6);
+ queueableReq.setCapability(Resource.newInstance(512, 3));
+ queueableReq.setPriority(Priority.newInstance(100));
+ queueableReq.setRelaxLocality(true);
+ queueableReq.setResourceName("*");
+ allocateRequest.setAskList(Arrays.asList(guaranteedReq, queueableReq));
+
+ // Verify 6 containers were allocated
+ allocateResponse = localScheduler.allocate(allocateRequest);
+ Assert.assertEquals(6, allocateResponse.getAllocatedContainers().size());
+
+ // Verify New containers are equally distribution on hosts c and d
+ // And None on a and b
+ allocs = mapAllocs(allocateResponse);
+ Assert.assertEquals(3, allocs.get(NodeId.newInstance("c", 3)).size());
+ Assert.assertEquals(3, allocs.get(NodeId.newInstance("d", 4)).size());
+ Assert.assertNull(allocs.get(NodeId.newInstance("a", 1)));
+ Assert.assertNull(allocs.get(NodeId.newInstance("b", 2)));
+ }
+
+ private DistSchedAllocateResponse createAllocateResponse(List nodes) {
+ DistSchedAllocateResponse distSchedAllocateResponse = Records.newRecord
+ (DistSchedAllocateResponse.class);
+ distSchedAllocateResponse.setAllocateResponse(
+ Records.newRecord(AllocateResponse.class));
+ distSchedAllocateResponse.setNodesForScheduling(nodes);
+ return distSchedAllocateResponse;
+ }
+
+ private Map> mapAllocs(AllocateResponse
+ allocateResponse) {
+ Map> allocs = new HashMap<>();
+ for (Container c : allocateResponse.getAllocatedContainers()) {
+ List cIds = allocs.get(c.getNodeId());
+ if (cIds == null) {
+ cIds = new ArrayList<>();
+ allocs.put(c.getNodeId(), cIds);
+ }
+ cIds.add(c.getId());
+ }
+ return allocs;
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
index 84e42fc..6a72cc0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
@@ -96,7 +96,7 @@ public void testContainerLogDirs() throws IOException, YarnException {
healthChecker.init(conf);
LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
NMContext nmContext = new NodeManager.NMContext(null, null, dirsHandler,
- new ApplicationACLsManager(conf), new NMNullStateStoreService());
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false);
// Add an application and the corresponding containers
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(conf);
String user = "nobody";
@@ -136,7 +136,7 @@ public void testContainerLogDirs() throws IOException, YarnException {
when(dirsHandlerForFullDisk.getLogDirsForRead()).
thenReturn(Arrays.asList(new String[] {absLogDir.getAbsolutePath()}));
nmContext = new NodeManager.NMContext(null, null, dirsHandlerForFullDisk,
- new ApplicationACLsManager(conf), new NMNullStateStoreService());
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false);
nmContext.getApplications().put(appId, app);
container.setState(ContainerState.RUNNING);
nmContext.getContainers().put(container1, container);
@@ -158,7 +158,7 @@ public void testContainerLogFile() throws IOException, YarnException {
LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
dirsHandler.init(conf);
NMContext nmContext = new NodeManager.NMContext(null, null, dirsHandler,
- new ApplicationACLsManager(conf), new NMNullStateStoreService());
+ new ApplicationACLsManager(conf), new NMNullStateStoreService(), false);
// Add an application and the corresponding containers
String user = "nobody";
long clusterTimeStamp = 1234;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
index e1845c7..39e8394 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
@@ -87,7 +87,7 @@ private NodeHealthCheckerService createNodeHealthCheckerService(Configuration co
private int startNMWebAppServer(String webAddr) {
Context nmContext = new NodeManager.NMContext(null, null, null, null,
- null);
+ null, false);
ResourceView resourceView = new ResourceView() {
@Override
public long getVmemAllocatedForContainers() {
@@ -150,7 +150,7 @@ public void testNMWebAppWithEphemeralPort() throws IOException {
@Test
public void testNMWebApp() throws IOException, YarnException {
Context nmContext = new NodeManager.NMContext(null, null, null, null,
- null);
+ null, false);
ResourceView resourceView = new ResourceView() {
@Override
public long getVmemAllocatedForContainers() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index 1f5590c..2ac0956 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -111,7 +111,7 @@ protected void configureServlets() {
healthChecker.init(conf);
aclsManager = new ApplicationACLsManager(conf);
nmContext = new NodeManager.NMContext(null, null, dirsHandler,
- aclsManager, null);
+ aclsManager, null, false);
NodeId nodeId = NodeId.newInstance("testhost.foo.com", 8042);
((NodeManager.NMContext)nmContext).setNodeId(nodeId);
resourceView = new ResourceView() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
index e274abb..dfbcf06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
@@ -104,7 +104,7 @@ protected void configureServlets() {
dirsHandler = healthChecker.getDiskHandler();
aclsManager = new ApplicationACLsManager(conf);
nmContext = new NodeManager.NMContext(null, null, dirsHandler,
- aclsManager, null);
+ aclsManager, null, false);
NodeId nodeId = NodeId.newInstance("testhost.foo.com", 9999);
((NodeManager.NMContext)nmContext).setNodeId(nodeId);
resourceView = new ResourceView() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
index 0ed56d3..efad825 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java
@@ -132,7 +132,7 @@ public boolean isPmemCheckEnabled() {
dirsHandler = healthChecker.getDiskHandler();
aclsManager = new ApplicationACLsManager(conf);
nmContext = new NodeManager.NMContext(null, null, dirsHandler,
- aclsManager, null) {
+ aclsManager, null, false) {
public NodeId getNodeId() {
return NodeId.newInstance("testhost.foo.com", 8042);
};
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index ab94175..6c015a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -35,6 +35,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.UserGroupInformation;
@@ -48,6 +49,9 @@
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb
+ .DistSchedRegisterResponsePBImpl;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -86,9 +90,15 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+ .AbstractYarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+ .SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security
+ .AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
import org.apache.hadoop.yarn.server.security.MasterKeyData;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -104,21 +114,27 @@
private static final Log LOG = LogFactory.getLog(ApplicationMasterService.class);
private final AMLivelinessMonitor amLivelinessMonitor;
private YarnScheduler rScheduler;
- private InetSocketAddress masterServiceAddress;
- private Server server;
- private final RecordFactory recordFactory =
+ protected InetSocketAddress masterServiceAddress;
+ protected Server server;
+ protected final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private final ConcurrentMap responseMap =
new ConcurrentHashMap();
- private final RMContext rmContext;
+ protected final RMContext rmContext;
- public ApplicationMasterService(RMContext rmContext, YarnScheduler scheduler) {
+ public ApplicationMasterService(String name, RMContext rmContext,
+ YarnScheduler scheduler) {
super(ApplicationMasterService.class.getName());
this.amLivelinessMonitor = rmContext.getAMLivelinessMonitor();
this.rScheduler = scheduler;
this.rmContext = rmContext;
}
+ public ApplicationMasterService(RMContext rmContext,
+ YarnScheduler scheduler) {
+ this(ApplicationMasterService.class.getName(), rmContext, scheduler);
+ }
+
@Override
protected void serviceInit(Configuration conf) throws Exception {
masterServiceAddress = conf.getSocketAddr(
@@ -139,11 +155,8 @@ protected void serviceStart() throws Exception {
serverConf.set(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
SaslRpcServer.AuthMethod.TOKEN.toString());
- this.server =
- rpc.getServer(ApplicationMasterProtocol.class, this, masterServiceAddress,
- serverConf, this.rmContext.getAMRMTokenSecretManager(),
- serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
- YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
+ this.server = getServer(rpc, serverConf, masterServiceAddress,
+ this.rmContext.getAMRMTokenSecretManager());
// Enable service authorization?
if (conf.getBoolean(
@@ -158,7 +171,7 @@ protected void serviceStart() throws Exception {
}
refreshServiceAcls(conf, RMPolicyProvider.getInstance());
}
-
+
this.server.start();
this.masterServiceAddress =
conf.updateConnectAddr(YarnConfiguration.RM_BIND_HOST,
@@ -168,6 +181,14 @@ protected void serviceStart() throws Exception {
super.serviceStart();
}
+ protected Server getServer(YarnRPC rpc, Configuration serverConf,
+ InetSocketAddress addr, AMRMTokenSecretManager secretManager) {
+ return rpc.getServer(ApplicationMasterProtocol.class, this, addr,
+ serverConf, secretManager,
+ serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
+ }
+
@Private
public InetSocketAddress getBindAddress() {
return this.masterServiceAddress;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java
new file mode 100644
index 0000000..9035990
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DistributedSchedulingService.java
@@ -0,0 +1,152 @@
+/**
+ * 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.server.resourcemanager;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
+
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.security
+ .AMRMTokenSecretManager;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+
+public class DistributedSchedulingService extends ApplicationMasterService
+ implements DistributedSchedulerProtocol {
+
+ public DistributedSchedulingService(RMContext rmContext,
+ YarnScheduler scheduler) {
+ super(DistributedSchedulingService.class.getName(), rmContext, scheduler);
+ }
+
+ @Override
+ public Server getServer(YarnRPC rpc, Configuration serverConf,
+ InetSocketAddress addr, AMRMTokenSecretManager secretManager) {
+ Server server = rpc.getServer(DistributedSchedulerProtocol.class, this,
+ addr, serverConf, secretManager,
+ serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
+ // To support application running no NMs that DO NOT support
+ // Dist Scheduling...
+ ((RPC.Server) server).addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+ ApplicationMasterProtocolPB.class,
+ ApplicationMasterProtocolService.newReflectiveBlockingService(
+ new ApplicationMasterProtocolPBServiceImpl(this)));
+ return server;
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return super.registerApplicationMaster(request);
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ return super.finishApplicationMaster(request);
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ return super.allocate(request);
+ }
+
+ @Override
+ public DistSchedRegisterResponse
+ registerApplicationMasterForDistributedScheduling(
+ RegisterApplicationMasterRequest request) throws YarnException,
+ IOException {
+ RegisterApplicationMasterResponse response =
+ registerApplicationMaster(request);
+ DistSchedRegisterResponse dsResp = recordFactory
+ .newRecordInstance(DistSchedRegisterResponse.class);
+ dsResp.setRegisterResponse(response);
+ dsResp.setMinAllocatableCapabilty(
+ Resource.newInstance(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MIN_MEMORY,
+ YarnConfiguration.DIST_SCHEDULING_MIN_MEMORY_DEFAULT),
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MIN_VCORES,
+ YarnConfiguration.DIST_SCHEDULING_MIN_VCORES_DEFAULT)
+ )
+ );
+ dsResp.setMaxAllocatableCapabilty(
+ Resource.newInstance(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY,
+ YarnConfiguration.DIST_SCHEDULING_MAX_MEMORY_DEFAULT),
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_MAX_VCORES,
+ YarnConfiguration.DIST_SCHEDULING_MAX_VCORES_DEFAULT)
+ )
+ );
+ dsResp.setContainerTokenExpiryInterval(
+ getConfig().getInt(
+ YarnConfiguration.DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS,
+ YarnConfiguration.
+ DIST_SCHEDULING_CONTAINER_TOKEN_EXPIRY_MS_DEFAULT));
+ dsResp.setContainerIdStart(
+ this.rmContext.getEpoch() << ResourceManager.EPOCH_BIT_SHIFT);
+
+ // Set nodes to be used for scheduling
+ // TODO: The actual computation of the list will happen in YARN-4412
+ // TODO: Till then, send the complete list
+ dsResp.setNodesForScheduling(
+ new ArrayList<>(this.rmContext.getRMNodes().keySet()));
+ return dsResp;
+ }
+
+ @Override
+ public DistSchedAllocateResponse allocateForDistributedScheduling
+ (AllocateRequest request) throws YarnException, IOException {
+ AllocateResponse response = allocate(request);
+ DistSchedAllocateResponse dsResp = recordFactory.newRecordInstance
+ (DistSchedAllocateResponse.class);
+ dsResp.setAllocateResponse(response);
+ dsResp.setNodesForScheduling(
+ new ArrayList<>(this.rmContext.getRMNodes().keySet()));
+ return dsResp;
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 01a1c8f..4e26257 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -120,6 +120,11 @@
*/
public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+ /**
+ * Used for generation of various ids.
+ */
+ public static final int EPOCH_BIT_SHIFT = 40;
+
private static final Log LOG = LogFactory.getLog(ResourceManager.class);
private static long clusterTimeStamp = System.currentTimeMillis();
@@ -1116,6 +1121,11 @@ protected ClientRMService createClientRMService() {
}
protected ApplicationMasterService createApplicationMasterService() {
+ if (this.rmContext.getYarnConfiguration().getBoolean(
+ YarnConfiguration.DIST_SCHEDULING_ENABLED,
+ YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT)) {
+ return new DistributedSchedulingService(this.rmContext, scheduler);
+ }
return new ApplicationMasterService(this.rmContext, scheduler);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index c5f8cd1..ba1adc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -42,6 +42,7 @@
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@@ -63,7 +64,6 @@
final String user;
// TODO making containerIdCounter long
private final AtomicLong containerIdCounter;
- private final int EPOCH_BIT_SHIFT = 40;
final Set priorities = new TreeSet(
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
@@ -82,7 +82,7 @@
boolean pending = true; // for app metrics
private ResourceUsage appResourceUsage;
-
+
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
String user, Queue queue, ActiveUsersManager activeUsersManager,
long epoch, ResourceUsage appResourceUsage) {
@@ -92,7 +92,8 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
this.queueName = queue.getQueueName();
this.user = user;
this.activeUsersManager = activeUsersManager;
- this.containerIdCounter = new AtomicLong(epoch << EPOCH_BIT_SHIFT);
+ this.containerIdCounter =
+ new AtomicLong(epoch << ResourceManager.EPOCH_BIT_SHIFT);
this.appResourceUsage = appResourceUsage;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index c5f8def..20e6066 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -41,6 +41,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
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.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId;
@@ -498,7 +499,8 @@ private Container updateContainerAndNMToken(RMContainer rmContainer,
.createContainerToken(container.getId(), container.getNodeId(),
getUser(), container.getResource(), container.getPriority(),
rmContainer.getCreationTime(), this.logAggregationContext,
- rmContainer.getNodeLabelExpression(), containerType));
+ rmContainer.getNodeLabelExpression(), containerType,
+ ExecutionType.GUARANTEED));
NMToken nmToken =
rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
getApplicationAttemptId(), container);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 6f00615..e859e70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
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.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
@@ -180,7 +181,8 @@ public Token createContainerToken(ContainerId containerId, NodeId nodeId,
String appSubmitter, Resource capability, Priority priority,
long createTime) {
return createContainerToken(containerId, nodeId, appSubmitter, capability,
- priority, createTime, null, null, ContainerType.TASK);
+ priority, createTime, null, null, ContainerType.TASK,
+ ExecutionType.GUARANTEED);
}
/**
@@ -195,12 +197,14 @@ public Token createContainerToken(ContainerId containerId, NodeId nodeId,
* @param logAggregationContext
* @param nodeLabelExpression
* @param containerType
+ * @param executionType
* @return the container-token
*/
public Token createContainerToken(ContainerId containerId, NodeId nodeId,
String appSubmitter, Resource capability, Priority priority,
long createTime, LogAggregationContext logAggregationContext,
- String nodeLabelExpression, ContainerType containerType) {
+ String nodeLabelExpression, ContainerType containerType,
+ ExecutionType executionType) {
byte[] password;
ContainerTokenIdentifier tokenIdentifier;
long expiryTimeStamp =
@@ -209,12 +213,12 @@ public Token createContainerToken(ContainerId containerId, NodeId nodeId,
// Lock so that we use the same MasterKey's keyId and its bytes
this.readLock.lock();
try {
- tokenIdentifier =
- new ContainerTokenIdentifier(containerId, nodeId.toString(),
- appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
- .getMasterKey().getKeyId(),
- ResourceManager.getClusterTimeStamp(), priority, createTime,
- logAggregationContext, nodeLabelExpression, containerType);
+ tokenIdentifier = new ContainerTokenIdentifier(containerId,
+ nodeId.toString(), appSubmitter, capability, expiryTimeStamp,
+ this.currentMasterKey.getMasterKey().getKeyId(),
+ ResourceManager.getClusterTimeStamp(), priority, createTime,
+ logAggregationContext, nodeLabelExpression, containerType,
+ executionType);
password = this.createPassword(tokenIdentifier);
} finally {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 0372cd7..bd1f15c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -91,6 +91,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+
+
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.YarnVersionInfo;
import org.apache.log4j.Level;
@@ -702,6 +704,21 @@ protected void serviceStop() {
@Override
protected ApplicationMasterService createApplicationMasterService() {
+ if (this.rmContext.getYarnConfiguration().getBoolean(
+ YarnConfiguration.DIST_SCHEDULING_ENABLED,
+ YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT)) {
+ return new DistributedSchedulingService(getRMContext(), scheduler) {
+ @Override
+ protected void serviceStart() {
+ // override to not start rpc handler
+ }
+
+ @Override
+ protected void serviceStop() {
+ // don't do anything
+ }
+ };
+ }
return new ApplicationMasterService(getRMContext(), scheduler) {
@Override
protected void serviceStart() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
new file mode 100644
index 0000000..a52e3aa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDistributedSchedulingService.java
@@ -0,0 +1,170 @@
+/**
+ * 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.server.resourcemanager;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocol;
+import org.apache.hadoop.yarn.api.DistributedSchedulerProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedAllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.DistSchedRegisterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
+ .AMLivelinessMonitor;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+
+public class TestDistributedSchedulingService {
+
+ // Test if the DistributedSchedulingService can handle both DSProtocol as
+ // well as AMProtocol clients
+ @Test
+ public void testRPCWrapping() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
+ .getName());
+ YarnRPC rpc = YarnRPC.create(conf);
+ String bindAddr = "localhost:0";
+ InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+ conf.setSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS, addr);
+ final RecordFactory factory = RecordFactoryProvider.getRecordFactory(null);
+ final RMContext rmContext = new RMContextImpl() {
+ @Override
+ public AMLivelinessMonitor getAMLivelinessMonitor() {
+ return null;
+ }
+ };
+ DistributedSchedulingService service =
+ new DistributedSchedulingService(rmContext, null) {
+ @Override
+ public RegisterApplicationMasterResponse registerApplicationMaster
+ (RegisterApplicationMasterRequest request) throws
+ YarnException, IOException {
+ RegisterApplicationMasterResponse resp = factory.newRecordInstance(
+ RegisterApplicationMasterResponse.class);
+ // Dummy Entry to Assert that we get this object back
+ resp.setQueue("dummyQueue");
+ return resp;
+ }
+
+ @Override
+ public FinishApplicationMasterResponse finishApplicationMaster
+ (FinishApplicationMasterRequest request) throws YarnException,
+ IOException {
+ FinishApplicationMasterResponse resp = factory.newRecordInstance(
+ FinishApplicationMasterResponse.class);
+ // Dummy Entry to Assert that we get this object back
+ resp.setIsUnregistered(false);
+ return resp;
+ }
+
+ @Override
+ public AllocateResponse allocate(AllocateRequest request) throws
+ YarnException, IOException {
+ AllocateResponse response = factory.newRecordInstance
+ (AllocateResponse.class);
+ response.setNumClusterNodes(12345);
+ return response;
+ }
+
+ @Override
+ public DistSchedRegisterResponse
+ registerApplicationMasterForDistributedScheduling
+ (RegisterApplicationMasterRequest request) throws
+ YarnException, IOException {
+ DistSchedRegisterResponse resp = factory.newRecordInstance(
+ DistSchedRegisterResponse.class);
+ resp.setContainerIdStart(54321l);
+ return resp;
+ }
+
+ @Override
+ public DistSchedAllocateResponse allocateForDistributedScheduling
+ (AllocateRequest request) throws YarnException, IOException {
+ DistSchedAllocateResponse resp =
+ factory.newRecordInstance(DistSchedAllocateResponse.class);
+ resp.setNodesForScheduling(
+ Arrays.asList(NodeId.newInstance("h1", 1234)));
+ return resp;
+ }
+ };
+ Server server = service.getServer(rpc, conf, addr, null);
+ server.start();
+
+ // Verify that the DistrubutedSchedulingService can handle vanilla
+ // ApplicationMasterProtocol clients
+ RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class,
+ ProtobufRpcEngine.class);
+ ApplicationMasterProtocol ampProxy =
+ (ApplicationMasterProtocol) rpc.getProxy(ApplicationMasterProtocol
+ .class, NetUtils.getConnectAddress(server), conf);
+ RegisterApplicationMasterResponse regResp = ampProxy.registerApplicationMaster(
+ factory.newRecordInstance(RegisterApplicationMasterRequest.class));
+ Assert.assertEquals("dummyQueue", regResp.getQueue());
+ FinishApplicationMasterResponse finishResp = ampProxy
+ .finishApplicationMaster(factory.newRecordInstance(
+ FinishApplicationMasterRequest.class));
+ Assert.assertEquals(false, finishResp.getIsUnregistered());
+ AllocateResponse allocResp = ampProxy
+ .allocate(factory.newRecordInstance(AllocateRequest.class));
+ Assert.assertEquals(12345, allocResp.getNumClusterNodes());
+
+
+ // Verify that the DistrubutedSchedulingService can handle the
+ // DistributedSchedulerProtocol clients as well
+ RPC.setProtocolEngine(conf, DistributedSchedulerProtocolPB.class,
+ ProtobufRpcEngine.class);
+ DistributedSchedulerProtocol dsProxy =
+ (DistributedSchedulerProtocol) rpc.getProxy(DistributedSchedulerProtocol
+ .class, NetUtils.getConnectAddress(server), conf);
+
+ DistSchedRegisterResponse dsRegResp =
+ dsProxy.registerApplicationMasterForDistributedScheduling(
+ factory.newRecordInstance(RegisterApplicationMasterRequest.class));
+ Assert.assertEquals(54321l, dsRegResp.getContainerIdStart());
+ DistSchedAllocateResponse dsAllocResp =
+ dsProxy.allocateForDistributedScheduling(
+ factory.newRecordInstance(AllocateRequest.class));
+ Assert.assertEquals(
+ "h1", dsAllocResp.getNodesForScheduling().get(0).getHost());
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index b5b2222..5ffe14e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -28,6 +28,7 @@
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.Container;
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.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
@@ -290,11 +291,12 @@ protected RMContainerTokenSecretManager createContainerTokenSecretManager(
public Token createContainerToken(ContainerId containerId,
NodeId nodeId, String appSubmitter, Resource capability,
Priority priority, long createTime,
- LogAggregationContext logAggregationContext, String nodeLabelExp, ContainerType containerType) {
+ LogAggregationContext logAggregationContext, String nodeLabelExp,
+ ContainerType containerType, ExecutionType executionType) {
numRetries++;
return super.createContainerToken(containerId, nodeId, appSubmitter,
capability, priority, createTime, logAggregationContext,
- nodeLabelExp, containerType);
+ nodeLabelExp, containerType, executionType);
}
};
}