diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java index 41ee65d..6c3a4d6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java @@ -30,6 +30,8 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.junit.Assert; import org.apache.commons.logging.Log; @@ -449,5 +451,14 @@ public StopContainersResponse stopContainers(StopContainersRequest request) "Dummy function cause")); throw new IOException(e); } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws IOException, + IOException { + Exception e = new Exception("Dummy function", new Exception( + "Dummy function cause")); + throw new IOException(e); + } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java index 184f1b2..610448c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java @@ -46,6 +46,8 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; @@ -454,6 +456,13 @@ public GetContainerStatusesResponse getContainerStatuses( } @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, + IOException { + return null; + } + + @Override public void close() throws IOException { } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java index f901ed8..167d804 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java @@ -46,8 +46,6 @@ 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.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NodeReport; @@ -254,8 +252,8 @@ public AllocateResponse allocate(AllocateRequest request) Resources.none(), null, 1, null, Collections.emptyList(), yarnToken, - Collections.emptyList(), - Collections.emptyList()); + Collections.emptyList(), + Collections.emptyList()); } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java index e148c32..2bb7e27 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java @@ -98,6 +98,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NMToken; @@ -1575,8 +1576,10 @@ public MyFifoScheduler(RMContext rmContext) { @Override public synchronized Allocation allocate( ApplicationAttemptId applicationAttemptId, List ask, - List release, - List blacklistAdditions, List blacklistRemovals) { + List release, List blacklistAdditions, + List blacklistRemovals, + List increaseRequests, + List decreaseRequests) { List askCopy = new ArrayList(); for (ResourceRequest req : ask) { ResourceRequest reqCopy = ResourceRequest.newInstance(req @@ -1590,8 +1593,8 @@ public synchronized Allocation allocate( lastBlacklistAdditions = blacklistAdditions; lastBlacklistRemovals = blacklistRemovals; return super.allocate( - applicationAttemptId, askCopy, release, - blacklistAdditions, blacklistRemovals); + applicationAttemptId, askCopy, release, blacklistAdditions, + blacklistRemovals, increaseRequests, decreaseRequests); } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java index 2d2c3e0..dae2ce7 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java @@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.net.Node; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; @@ -174,6 +175,19 @@ public String getNodeManagerVersion() { public Set getNodeLabels() { return RMNodeLabelsManager.EMPTY_STRING_SET; } + + @Override + public void updateNodeHeartbeatResponseForContainersDecreasing( + NodeHeartbeatResponse response) { + // TODO Auto-generated method stub + + } + + @Override + public List pullNewlyIncreasedContainers() { + // TODO Auto-generated method stub + return null; + } } public static RMNode newNodeInfo(String rackName, String hostName, diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java index ecc4734..8c65ccc 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.net.Node; 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.NodeId; import org.apache.hadoop.yarn.api.records.NodeState; @@ -163,4 +164,16 @@ public String getNodeManagerVersion() { public Set getNodeLabels() { return RMNodeLabelsManager.EMPTY_STRING_SET; } + + @Override + public void updateNodeHeartbeatResponseForContainersDecreasing( + NodeHeartbeatResponse response) { + // TODO Auto-generated method stub + } + + @Override + public List pullNewlyIncreasedContainers() { + // TODO Auto-generated method stub + return null; + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java index 14e2645..310b3b5 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java @@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; @@ -202,15 +204,16 @@ public void run() { @Override public Allocation allocate(ApplicationAttemptId attemptId, - List resourceRequests, - List containerIds, - List strings, List strings2) { + List resourceRequests, List containerIds, + List strings, List strings2, + List increaseRequests, + List decreaseRequests) { if (metricsON) { final Timer.Context context = schedulerAllocateTimer.time(); Allocation allocation = null; try { allocation = scheduler.allocate(attemptId, resourceRequests, - containerIds, strings, strings2); + containerIds, strings, strings2, null, null); return allocation; } finally { context.stop(); @@ -224,7 +227,7 @@ public Allocation allocate(ApplicationAttemptId attemptId, } } else { return scheduler.allocate(attemptId, - resourceRequests, containerIds, strings, strings2); + resourceRequests, containerIds, strings, strings2, null, null); } } @@ -959,4 +962,12 @@ public Priority checkAndGetApplicationPriority(Priority priority, return Priority.newInstance(0); } + @Override + protected void decreaseContainer( + SchedContainerChangeRequest decreaseRequest, + SchedulerApplicationAttempt attempt) { + // TODO Auto-generated method stub + + } + } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java index a4416db..3626027 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java @@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -176,15 +177,17 @@ public void run() { @Override public Allocation allocate(ApplicationAttemptId attemptId, - List resourceRequests, - List containerIds, - List strings, List strings2) { + List resourceRequests, List containerIds, + List strings, List strings2, + List increaseRequests, + List decreaseRequests) { if (metricsON) { final Timer.Context context = schedulerAllocateTimer.time(); Allocation allocation = null; try { - allocation = super.allocate(attemptId, resourceRequests, - containerIds, strings, strings2); + allocation = super + .allocate(attemptId, resourceRequests, containerIds, strings, + strings2, increaseRequests, decreaseRequests); return allocation; } finally { context.stop(); @@ -197,8 +200,8 @@ public Allocation allocate(ApplicationAttemptId attemptId, } } } else { - return super.allocate(attemptId, - resourceRequests, containerIds, strings, strings2); + return super.allocate(attemptId, resourceRequests, containerIds, strings, + strings2, increaseRequests, decreaseRequests); } } @@ -426,7 +429,7 @@ private void tearDown() throws IOException { if (pool != null) pool.shutdown(); } - @SuppressWarnings("unchecked") + @SuppressWarnings({ "unchecked", "rawtypes" }) private void initMetrics() throws Exception { metrics = new MetricRegistry(); // configuration diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index b4c5c5e..ed1a276 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -194,6 +194,29 @@ Release 2.8.0 - UNRELEASED command line. (Inigo Goiri, Kenji Kikushima and Junping Du via junping_du) + YARN-3866. AM-RM protocol changes to support container resizing. (Meng Ding + via jianhe) + + YARN-1449. AM-NM protocol changes to support container resizing. + (Meng Ding & Wangda Tan via jianhe) + + YARN-1645. ContainerManager implementation to support container resizing. + (Meng Ding & Wangda Tan via jianhe) + + YARN-3867. ContainerImpl changes to support container resizing. (Meng Ding + via jianhe) + + YARN-1643. Make ContainersMonitor support changing monitoring size of an + allocated container. (Meng Ding and Wangda Tan) + + YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to + support container resizing. (Meng Ding via jianhe) + + YARN-3868. Recovery support for container resizing. (Meng Ding via jianhe) + + YARN-1651. CapacityScheduler side changes to support container resize. + (Wangda Tan via jianhe) + IMPROVEMENTS YARN-644. Basic null check is not performed on passed in arguments before diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java index 7aa43df..43e1d4c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java @@ -22,6 +22,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -38,9 +41,9 @@ /** *

The protocol between an ApplicationMaster and a - * NodeManager to start/stop containers and to get status - * of running containers.

- * + * NodeManager to start/stop and increase resource of containers + * and to get status of running containers.

+ * *

If security is enabled the NodeManager verifies that the * ApplicationMaster has truly been allocated the container * by the ResourceManager and also verifies all interactions such @@ -170,4 +173,25 @@ StopContainersResponse stopContainers(StopContainersRequest request) GetContainerStatusesResponse getContainerStatuses( GetContainerStatusesRequest request) throws YarnException, IOException; + + /** + *

+ * The API used by the ApplicationMaster to request for + * resource increase of running containers on the NodeManager. + *

+ * + * @param request + * request to increase resource of a list of containers + * @return response which includes a list of containerIds of containers + * whose resource has been successfully increased and a + * containerId-to-exception map for failed requests. + * + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, + IOException; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java index 2458d9b..0b65e5c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java @@ -22,11 +22,12 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.util.Records; @@ -46,6 +47,16 @@ *
  • * A list of unused {@link Container} which are being returned. *
  • + *
  • + * A list of {@link ContainerResourceChangeRequest} to inform + * the ResourceManager about the resource increase + * requirements of running containers. + *
  • + *
  • + * A list of {@link ContainerResourceChangeRequest} to inform + * the ResourceManager about the resource decrease + * requirements of running containers. + *
  • * * * @see ApplicationMasterProtocol#allocate(AllocateRequest) @@ -61,7 +72,7 @@ public static AllocateRequest newInstance(int responseID, float appProgress, List containersToBeReleased, ResourceBlacklistRequest resourceBlacklistRequest) { return newInstance(responseID, appProgress, resourceAsk, - containersToBeReleased, resourceBlacklistRequest, null); + containersToBeReleased, resourceBlacklistRequest, null, null); } @Public @@ -70,7 +81,8 @@ public static AllocateRequest newInstance(int responseID, float appProgress, List resourceAsk, List containersToBeReleased, ResourceBlacklistRequest resourceBlacklistRequest, - List increaseRequests) { + List increaseRequests, + List decreaseRequests) { AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class); allocateRequest.setResponseId(responseID); allocateRequest.setProgress(appProgress); @@ -78,6 +90,7 @@ public static AllocateRequest newInstance(int responseID, float appProgress, allocateRequest.setReleaseList(containersToBeReleased); allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest); allocateRequest.setIncreaseRequests(increaseRequests); + allocateRequest.setDecreaseRequests(decreaseRequests); return allocateRequest; } @@ -184,20 +197,38 @@ public abstract void setResourceBlacklistRequest( ResourceBlacklistRequest resourceBlacklistRequest); /** - * Get the ContainerResourceIncreaseRequest being sent by the - * ApplicationMaster + * Get the list of container resource increase requests being sent by the + * ApplicationMaster. */ @Public - @Stable - public abstract List getIncreaseRequests(); - + @Unstable + public abstract List getIncreaseRequests(); + /** - * Set the ContainerResourceIncreaseRequest to inform the - * ResourceManager about some container's resources need to be - * increased + * Set the list of container resource increase requests to inform the + * ResourceManager about the containers whose resources need + * to be increased. */ @Public - @Stable + @Unstable public abstract void setIncreaseRequests( - List increaseRequests); + List increaseRequests); + + /** + * Get the list of container resource decrease requests being sent by the + * ApplicationMaster. + */ + @Public + @Unstable + public abstract List getDecreaseRequests(); + + /** + * Set the list of container resource decrease requests to inform the + * ResourceManager about the containers whose resources need + * to be decreased. + */ + @Public + @Unstable + public abstract void setDecreaseRequests( + List decreaseRequests); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java index c4fdb79..c363070 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java @@ -28,8 +28,6 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.records.AMCommand; import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NodeReport; @@ -59,6 +57,14 @@ *
  • The number of available nodes in a cluster.
  • *
  • A description of resources requested back by the cluster
  • *
  • AMRMToken, if AMRMToken has been rolled over
  • + *
  • + * A list of {@link Container} representing the containers + * whose resource has been increased. + *
  • + *
  • + * A list of {@link Container} representing the containers + * whose resource has been decreased. + *
  • * * * @see ApplicationMasterProtocol#allocate(AllocateRequest) @@ -94,8 +100,8 @@ public static AllocateResponse newInstance(int responseId, List allocatedContainers, List updatedNodes, Resource availResources, AMCommand command, int numClusterNodes, PreemptionMessage preempt, List nmTokens, - List increasedContainers, - List decreasedContainers) { + List increasedContainers, + List decreasedContainers) { AllocateResponse response = newInstance(responseId, completedContainers, allocatedContainers, updatedNodes, availResources, command, numClusterNodes, preempt, nmTokens); @@ -111,8 +117,8 @@ public static AllocateResponse newInstance(int responseId, List allocatedContainers, List updatedNodes, Resource availResources, AMCommand command, int numClusterNodes, PreemptionMessage preempt, List nmTokens, Token amRMToken, - List increasedContainers, - List decreasedContainers) { + List increasedContainers, + List decreasedContainers) { AllocateResponse response = newInstance(responseId, completedContainers, allocatedContainers, updatedNodes, availResources, command, numClusterNodes, preempt, @@ -263,34 +269,38 @@ public static AllocateResponse newInstance(int responseId, public abstract void setNMTokens(List nmTokens); /** - * Get the list of newly increased containers by ResourceManager + * Get the list of newly increased containers by + * ResourceManager. */ @Public - @Stable - public abstract List getIncreasedContainers(); + @Unstable + public abstract List getIncreasedContainers(); /** - * Set the list of newly increased containers by ResourceManager + * Set the list of newly increased containers by + * ResourceManager. */ @Private @Unstable public abstract void setIncreasedContainers( - List increasedContainers); + List increasedContainers); /** - * Get the list of newly decreased containers by NodeManager + * Get the list of newly decreased containers by + * ResourceManager. */ @Public - @Stable - public abstract List getDecreasedContainers(); + @Unstable + public abstract List getDecreasedContainers(); /** - * Set the list of newly decreased containers by NodeManager + * Set the list of newly decreased containers by + * ResourceManager. */ @Private @Unstable public abstract void setDecreasedContainers( - List decreasedContainers); + List decreasedContainers); /** * The AMRMToken that belong to this attempt diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java new file mode 100644 index 0000000..1fe8e94 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java @@ -0,0 +1,75 @@ +/** + * 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 java.util.List; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.records.NMToken; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.util.Records; + +/** + *

    The request sent by Application Master to the + * Node Manager to change the resource quota of a container.

    + * + * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest) + */ +@Public +@Unstable +public abstract class IncreaseContainersResourceRequest { + @Public + @Unstable + public static IncreaseContainersResourceRequest newInstance( + List containersToIncrease) { + IncreaseContainersResourceRequest request = + Records.newRecord(IncreaseContainersResourceRequest.class); + request.setContainersToIncrease(containersToIncrease); + return request; + } + + /** + * Get a list of container tokens to be used for authorization during + * container resource increase. + *

    + * Note: {@link NMToken} will be used for authenticating communication with + * {@code NodeManager}. + * @return the list of container tokens to be used for authorization during + * container resource increase. + * @see NMToken + */ + @Public + @Unstable + public abstract List getContainersToIncrease(); + + /** + * Set container tokens to be used during container resource increase. + * The token is acquired from + * AllocateResponse.getIncreasedContainers. + * The token contains the container id and resource capability required for + * container resource increase. + * @param containersToIncrease the list of container tokens to be used + * for container resource increase. + */ + @Public + @Unstable + public abstract void setContainersToIncrease( + List containersToIncrease); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java new file mode 100644 index 0000000..aeb1e83 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java @@ -0,0 +1,93 @@ +/** + * 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.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.SerializedException; +import org.apache.hadoop.yarn.util.Records; + +import java.util.List; +import java.util.Map; + +/** + *

    + * The response sent by the NodeManager to the + * ApplicationMaster when asked to increase container resource. + *

    + * + * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest) + */ +@Public +@Unstable +public abstract class IncreaseContainersResourceResponse { + + @Private + @Unstable + public static IncreaseContainersResourceResponse newInstance( + List successfullyIncreasedContainers, + Map failedRequests) { + IncreaseContainersResourceResponse response = + Records.newRecord(IncreaseContainersResourceResponse.class); + response.setSuccessfullyIncreasedContainers( + successfullyIncreasedContainers); + response.setFailedRequests(failedRequests); + return response; + } + + /** + * Get the list of containerIds of containers whose resource + * have been successfully increased. + * + * @return the list of containerIds of containers whose resource have + * been successfully increased. + */ + @Public + @Unstable + public abstract List getSuccessfullyIncreasedContainers(); + + /** + * Set the list of containerIds of containers whose resource have + * been successfully increased. + */ + @Private + @Unstable + public abstract void setSuccessfullyIncreasedContainers( + List succeedIncreasedContainers); + + /** + * Get the containerId-to-exception map in which the exception indicates + * error from each container for failed requests. + */ + @Public + @Unstable + public abstract Map getFailedRequests(); + + /** + * Set the containerId-to-exception map in which the exception indicates + * error from each container for failed requests. + */ + @Private + @Unstable + public abstract void setFailedRequests( + Map failedRequests); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java new file mode 100644 index 0000000..117015b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.api.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; +import org.apache.hadoop.yarn.util.Records; + +/** + * {@code ContainerResourceChangeRequest} represents the request made by an + * application to the {@code ResourceManager} to change resource allocation of + * a running {@code Container}. + *

    + * It includes: + *

      + *
    • {@link ContainerId} for the container.
    • + *
    • + * {@link Resource} capability of the container after the resource change + * is completed. + *
    • + *
    + * + * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + */ +@Public +@Unstable +public abstract class ContainerResourceChangeRequest { + + @Public + @Unstable + public static ContainerResourceChangeRequest newInstance( + ContainerId existingContainerId, Resource targetCapability) { + ContainerResourceChangeRequest context = Records + .newRecord(ContainerResourceChangeRequest.class); + context.setContainerId(existingContainerId); + context.setCapability(targetCapability); + return context; + } + + /** + * Get the ContainerId of the container. + * @return ContainerId of the container + */ + @Public + @Unstable + public abstract ContainerId getContainerId(); + + /** + * Set the ContainerId of the container. + * @param containerId ContainerId of the container + */ + @Public + @Unstable + public abstract void setContainerId(ContainerId containerId); + + /** + * Get the Resource capability of the container. + * @return Resource capability of the container + */ + @Public + @Unstable + public abstract Resource getCapability(); + + /** + * Set the Resource capability of the container. + * @param capability Resource capability of the container + */ + @Public + @Unstable + public abstract void setCapability(Resource capability); + + @Override + public int hashCode() { + return getCapability().hashCode() + getContainerId().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ContainerResourceChangeRequest) { + ContainerResourceChangeRequest ctx = + (ContainerResourceChangeRequest) other; + + if (getContainerId() == null && ctx.getContainerId() != null) { + return false; + } else if (!getContainerId().equals(ctx.getContainerId())) { + return false; + } + + if (getCapability() == null && ctx.getCapability() != null) { + return false; + } else if (!getCapability().equals(ctx.getCapability())) { + return false; + } + + return true; + } else { + return false; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java deleted file mode 100644 index d766d92..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.api.records; - -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.yarn.util.Records; - -/** - * Used by Application Master to ask Node Manager reduce size of a specified - * container - */ -public abstract class ContainerResourceDecrease { - @Public - public static ContainerResourceDecrease newInstance( - ContainerId existingContainerId, Resource targetCapability) { - ContainerResourceDecrease context = Records - .newRecord(ContainerResourceDecrease.class); - context.setContainerId(existingContainerId); - context.setCapability(targetCapability); - return context; - } - - @Public - public abstract ContainerId getContainerId(); - - @Public - public abstract void setContainerId(ContainerId containerId); - - @Public - public abstract Resource getCapability(); - - @Public - public abstract void setCapability(Resource capability); - - @Override - public int hashCode() { - return getCapability().hashCode() + getContainerId().hashCode(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof ContainerResourceDecrease) { - ContainerResourceDecrease ctx = (ContainerResourceDecrease)other; - - if (getContainerId() == null && ctx.getContainerId() != null) { - return false; - } else if (!getContainerId().equals(ctx.getContainerId())) { - return false; - } - - if (getCapability() == null && ctx.getCapability() != null) { - return false; - } else if (!getCapability().equals(ctx.getCapability())) { - return false; - } - - return true; - } else { - return false; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java deleted file mode 100644 index f4c1560..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.api.records; - -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.yarn.util.Records; - -/** - * Represent a new increased container accepted by Resource Manager - */ -public abstract class ContainerResourceIncrease { - @Public - public static ContainerResourceIncrease newInstance( - ContainerId existingContainerId, Resource targetCapability, Token token) { - ContainerResourceIncrease context = Records - .newRecord(ContainerResourceIncrease.class); - context.setContainerId(existingContainerId); - context.setCapability(targetCapability); - context.setContainerToken(token); - return context; - } - - @Public - public abstract ContainerId getContainerId(); - - @Public - public abstract void setContainerId(ContainerId containerId); - - @Public - public abstract Resource getCapability(); - - @Public - public abstract void setCapability(Resource capability); - - @Public - public abstract Token getContainerToken(); - - @Public - public abstract void setContainerToken(Token token); - - @Override - public int hashCode() { - return getCapability().hashCode() + getContainerId().hashCode(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof ContainerResourceIncrease) { - ContainerResourceIncrease ctx = (ContainerResourceIncrease)other; - - if (getContainerId() == null && ctx.getContainerId() != null) { - return false; - } else if (!getContainerId().equals(ctx.getContainerId())) { - return false; - } - - if (getCapability() == null && ctx.getCapability() != null) { - return false; - } else if (!getCapability().equals(ctx.getCapability())) { - return false; - } - - return true; - } else { - return false; - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java deleted file mode 100644 index 9e3b640..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.yarn.api.records; - -import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.yarn.util.Records; - -/** - * Used by Application Master, send a container resource increase request to - * Resource Manager - */ -@Public -public abstract class ContainerResourceIncreaseRequest { - @Public - public static ContainerResourceIncreaseRequest newInstance( - ContainerId existingContainerId, Resource targetCapability) { - ContainerResourceIncreaseRequest context = Records - .newRecord(ContainerResourceIncreaseRequest.class); - context.setContainerId(existingContainerId); - context.setCapability(targetCapability); - return context; - } - - @Public - public abstract ContainerId getContainerId(); - - @Public - public abstract void setContainerId(ContainerId containerId); - - @Public - public abstract Resource getCapability(); - - @Public - public abstract void setCapability(Resource capability); - - @Override - public int hashCode() { - return getCapability().hashCode() + getContainerId().hashCode(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof ContainerResourceIncreaseRequest) { - ContainerResourceIncreaseRequest ctx = - (ContainerResourceIncreaseRequest) other; - - if (getContainerId() == null && ctx.getContainerId() != null) { - return false; - } else if (!getContainerId().equals(ctx.getContainerId())) { - return false; - } - - if (getCapability() == null && ctx.getCapability() != null) { - return false; - } else if (!getCapability().equals(ctx.getCapability())) { - return false; - } - - return true; - } else { - return false; - } - } -} 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 5ccf6dc..2c2238f 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 @@ -34,6 +34,7 @@ *
  • {@code ContainerState} of the container.
  • *
  • Exit status of a completed container.
  • *
  • Diagnostic message for a failed container.
  • + *
  • {@link Resource} allocated to the container.
  • * */ @Public @@ -114,4 +115,16 @@ public static ContainerStatus newInstance(ContainerId containerId, @Private @Unstable public abstract void setDiagnostics(String diagnostics); + + /** + * Get the Resource allocated to the container. + * @return Resource allocated to the container + */ + @Public + @Unstable + public abstract Resource getCapability(); + + @Private + @Unstable + public abstract void setCapability(Resource capability); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto index 7b1647b..f06f6cb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto @@ -34,4 +34,5 @@ service ContainerManagementProtocolService { rpc startContainers(StartContainersRequestProto) returns (StartContainersResponseProto); rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto); rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto); + rpc increaseContainersResource(IncreaseContainersResourceRequestProto) returns (IncreaseContainersResourceResponseProto); } 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 0bccfc4..057aeee 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 @@ -470,6 +470,7 @@ message ContainerStatusProto { optional ContainerStateProto state = 2; optional string diagnostics = 3 [default = "N/A"]; optional int32 exit_status = 4 [default = -1000]; + optional ResourceProto capability = 5; } enum ContainerExitStatusProto { @@ -479,22 +480,11 @@ enum ContainerExitStatusProto { DISKS_FAILED = -101; } -message ContainerResourceIncreaseRequestProto { +message ContainerResourceChangeRequestProto { optional ContainerIdProto container_id = 1; optional ResourceProto capability = 2; } -message ContainerResourceIncreaseProto { - optional ContainerIdProto container_id = 1; - optional ResourceProto capability = 2; - optional hadoop.common.TokenProto container_token = 3; -} - -message ContainerResourceDecreaseProto { - optional ContainerIdProto container_id = 1; - optional ResourceProto capability = 2; -} - //////////////////////////////////////////////////////////////////////// ////// From common////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////// 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 b0b12d1..ff5a127 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 @@ -66,7 +66,8 @@ message AllocateRequestProto { optional ResourceBlacklistRequestProto blacklist_request = 3; optional int32 response_id = 4; optional float progress = 5; - repeated ContainerResourceIncreaseRequestProto increase_request = 6; + repeated ContainerResourceChangeRequestProto increase_request = 6; + repeated ContainerResourceChangeRequestProto decrease_request = 7; } message NMTokenProto { @@ -84,8 +85,8 @@ message AllocateResponseProto { optional int32 num_cluster_nodes = 7; optional PreemptionMessageProto preempt = 8; repeated NMTokenProto nm_tokens = 9; - repeated ContainerResourceIncreaseProto increased_containers = 10; - repeated ContainerResourceDecreaseProto decreased_containers = 11; + repeated ContainerProto increased_containers = 10; + repeated ContainerProto decreased_containers = 11; optional hadoop.common.TokenProto am_rm_token = 12; } @@ -286,6 +287,15 @@ message GetContainerStatusesResponseProto { repeated ContainerExceptionMapProto failed_requests = 2; } +message IncreaseContainersResourceRequestProto { + repeated hadoop.common.TokenProto increase_containers = 1; +} + +message IncreaseContainersResourceResponseProto { + repeated ContainerIdProto succeeded_requests = 1; + repeated ContainerExceptionMapProto failed_requests = 2; +} + ////////////////////////////////////////////////////// /////// Application_History_Protocol ///////////////// ////////////////////////////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java index 6cdf87f..338198b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java @@ -68,7 +68,7 @@ public void testResourceTrackerOnHA() throws Exception { failoverThread = createAndStartFailoverThread(); NodeStatus status = NodeStatus.newInstance(NodeId.newInstance("localhost", 0), 0, null, - null, null, null, null); + null, null, null, null, null); NodeHeartbeatRequest request2 = NodeHeartbeatRequest.newInstance(status, null, null,null); resourceTracker.nodeHeartbeat(request2); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java index 108ad37..2394747 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java @@ -36,6 +36,7 @@ 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.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.Priority; @@ -525,7 +526,9 @@ public MyFifoScheduler(RMContext rmContext) { public synchronized Allocation allocate( ApplicationAttemptId applicationAttemptId, List ask, List release, List blacklistAdditions, - List blacklistRemovals) { + List blacklistRemovals, + List increaseRequests, + List decreaseRequests) { List askCopy = new ArrayList(); for (ResourceRequest req : ask) { ResourceRequest reqCopy = @@ -539,7 +542,8 @@ public synchronized Allocation allocate( lastBlacklistAdditions = blacklistAdditions; lastBlacklistRemovals = blacklistRemovals; return super.allocate(applicationAttemptId, askCopy, release, - blacklistAdditions, blacklistRemovals); + blacklistAdditions, blacklistRemovals, increaseRequests, + decreaseRequests); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java index 15397e3..ce18bde 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java @@ -30,12 +30,16 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl; @@ -48,6 +52,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto; import com.google.protobuf.ServiceException; @@ -128,4 +133,19 @@ public GetContainerStatusesResponse getContainerStatuses( return null; } } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, + IOException { + IncreaseContainersResourceRequestProto requestProto = + ((IncreaseContainersResourceRequestPBImpl)request).getProto(); + try { + return new IncreaseContainersResourceResponsePBImpl( + proxy.increaseContainersResource(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/ContainerManagementProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java index 2d33e69..7626441 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java @@ -23,9 +23,12 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse; import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl; @@ -33,6 +36,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto; @@ -94,4 +99,21 @@ public GetContainerStatusesResponseProto getContainerStatuses( throw new ServiceException(e); } } + + @Override + public IncreaseContainersResourceResponseProto increaseContainersResource( + RpcController controller, IncreaseContainersResourceRequestProto proto) + throws ServiceException { + IncreaseContainersResourceRequestPBImpl request = + new IncreaseContainersResourceRequestPBImpl(proto); + try { + IncreaseContainersResourceResponse response = + real.increaseContainersResource(request); + return ((IncreaseContainersResourceResponsePBImpl)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/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java index dc11165..d6db32c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java @@ -27,15 +27,15 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto; @@ -52,7 +52,8 @@ private List ask = null; private List release = null; - private List increaseRequests = null; + private List increaseRequests = null; + private List decreaseRequests = null; private ResourceBlacklistRequest blacklistRequest = null; public AllocateRequestPBImpl() { @@ -101,6 +102,9 @@ private void mergeLocalToBuilder() { if (this.increaseRequests != null) { addIncreaseRequestsToProto(); } + if (this.decreaseRequests != null) { + addDecreaseRequestsToProto(); + } if (this.blacklistRequest != null) { builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest)); } @@ -162,14 +166,14 @@ public void setAskList(final List resourceRequests) { } @Override - public List getIncreaseRequests() { + public List getIncreaseRequests() { initIncreaseRequests(); return this.increaseRequests; } @Override public void setIncreaseRequests( - List increaseRequests) { + List increaseRequests) { if (increaseRequests == null) { return; } @@ -177,7 +181,24 @@ public void setIncreaseRequests( this.increaseRequests.clear(); this.increaseRequests.addAll(increaseRequests); } - + + @Override + public List getDecreaseRequests() { + initDecreaseRequests(); + return this.decreaseRequests; + } + + @Override + public void setDecreaseRequests( + List decreaseRequests) { + if (decreaseRequests == null) { + return; + } + initDecreaseRequests(); + this.decreaseRequests.clear(); + this.decreaseRequests.addAll(decreaseRequests); + } + @Override public ResourceBlacklistRequest getResourceBlacklistRequest() { AllocateRequestProtoOrBuilder p = viaProto ? proto : builder; @@ -252,28 +273,42 @@ private void initIncreaseRequests() { return; } AllocateRequestProtoOrBuilder p = viaProto ? proto : builder; - List list = + List list = p.getIncreaseRequestList(); - this.increaseRequests = new ArrayList(); + this.increaseRequests = new ArrayList(); - for (ContainerResourceIncreaseRequestProto c : list) { + for (ContainerResourceChangeRequestProto c : list) { this.increaseRequests.add(convertFromProtoFormat(c)); } } - + + private void initDecreaseRequests() { + if (this.decreaseRequests != null) { + return; + } + AllocateRequestProtoOrBuilder p = viaProto ? proto : builder; + List list = + p.getDecreaseRequestList(); + this.decreaseRequests = new ArrayList<>(); + + for (ContainerResourceChangeRequestProto c : list) { + this.decreaseRequests.add(convertFromProtoFormat(c)); + } + } + private void addIncreaseRequestsToProto() { maybeInitBuilder(); builder.clearIncreaseRequest(); if (increaseRequests == null) { return; } - Iterable iterable = - new Iterable() { + Iterable iterable = + new Iterable() { @Override - public Iterator iterator() { - return new Iterator() { + public Iterator iterator() { + return new Iterator() { - Iterator iter = + Iterator iter = increaseRequests.iterator(); @Override @@ -282,7 +317,7 @@ public boolean hasNext() { } @Override - public ContainerResourceIncreaseRequestProto next() { + public ContainerResourceChangeRequestProto next() { return convertToProtoFormat(iter.next()); } @@ -296,7 +331,43 @@ public void remove() { }; builder.addAllIncreaseRequest(iterable); } - + + private void addDecreaseRequestsToProto() { + maybeInitBuilder(); + builder.clearDecreaseRequest(); + if (decreaseRequests == null) { + return; + } + Iterable iterable = + new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + Iterator iter = + decreaseRequests.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public ContainerResourceChangeRequestProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + + } + }; + builder.addAllDecreaseRequest(iterable); + } + @Override public List getReleaseList() { initReleases(); @@ -367,14 +438,14 @@ private ResourceRequestProto convertToProtoFormat(ResourceRequest t) { return ((ResourceRequestPBImpl)t).getProto(); } - private ContainerResourceIncreaseRequestPBImpl convertFromProtoFormat( - ContainerResourceIncreaseRequestProto p) { - return new ContainerResourceIncreaseRequestPBImpl(p); + private ContainerResourceChangeRequestPBImpl convertFromProtoFormat( + ContainerResourceChangeRequestProto p) { + return new ContainerResourceChangeRequestPBImpl(p); } - private ContainerResourceIncreaseRequestProto convertToProtoFormat( - ContainerResourceIncreaseRequest t) { - return ((ContainerResourceIncreaseRequestPBImpl) t).getProto(); + private ContainerResourceChangeRequestProto convertToProtoFormat( + ContainerResourceChangeRequest t) { + return ((ContainerResourceChangeRequestPBImpl) t).getProto(); } private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java index f2796fd..dd7d1a9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java @@ -29,8 +29,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.records.AMCommand; import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NodeReport; @@ -38,8 +36,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl; @@ -48,8 +44,6 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto; @@ -72,8 +66,8 @@ private List allocatedContainers = null; private List nmTokens = null; private List completedContainersStatuses = null; - private List increasedContainers = null; - private List decreasedContainers = null; + private List increasedContainers = null; + private List decreasedContainers = null; private List updatedNodes = null; private PreemptionMessage preempt; @@ -147,14 +141,14 @@ private synchronized void mergeLocalToBuilder() { } if (this.increasedContainers != null) { builder.clearIncreasedContainers(); - Iterable iterable = - getIncreaseProtoIterable(this.increasedContainers); + Iterable iterable = + getContainerProtoIterable(this.increasedContainers); builder.addAllIncreasedContainers(iterable); } if (this.decreasedContainers != null) { builder.clearDecreasedContainers(); - Iterable iterable = - getChangeProtoIterable(this.decreasedContainers); + Iterable iterable = + getContainerProtoIterable(this.decreasedContainers); builder.addAllDecreasedContainers(iterable); } if (this.amrmToken != null) { @@ -262,6 +256,36 @@ public synchronized void setAllocatedContainers( allocatedContainers.addAll(containers); } + @Override + public synchronized List getIncreasedContainers() { + initLocalIncreasedContainerList(); + return this.increasedContainers; + } + + @Override + public synchronized void setIncreasedContainers( + final List containers) { + if (containers == null) + return; + initLocalIncreasedContainerList(); + increasedContainers.addAll(containers); + } + + @Override + public synchronized List getDecreasedContainers() { + initLocalDecreasedContainerList(); + return this.decreasedContainers; + } + + @Override + public synchronized void setDecreasedContainers( + final List containers) { + if (containers == null) + return; + initLocalDecreasedContainerList(); + decreasedContainers.addAll(containers); + } + //// Finished containers @Override public synchronized List getCompletedContainersStatuses() { @@ -333,37 +357,6 @@ public synchronized void setPreemptionMessage(PreemptionMessage preempt) { } @Override - public synchronized List getIncreasedContainers() { - initLocalIncreasedContainerList(); - return increasedContainers; - } - - @Override - public synchronized void setIncreasedContainers( - List increasedContainers) { - if (increasedContainers == null) - return; - initLocalIncreasedContainerList(); - this.increasedContainers.addAll(increasedContainers); - } - - @Override - public synchronized List getDecreasedContainers() { - initLocalDecreasedContainerList(); - return decreasedContainers; - } - - @Override - public synchronized void setDecreasedContainers( - List decreasedContainers) { - if (decreasedContainers == null) { - return; - } - initLocalDecreasedContainerList(); - this.decreasedContainers.addAll(decreasedContainers); - } - - @Override public synchronized Token getAMRMToken() { AllocateResponseProtoOrBuilder p = viaProto ? proto : builder; if (amrmToken != null) { @@ -390,10 +383,10 @@ private synchronized void initLocalIncreasedContainerList() { return; } AllocateResponseProtoOrBuilder p = viaProto ? proto : builder; - List list = p.getIncreasedContainersList(); - increasedContainers = new ArrayList(); + List list = p.getIncreasedContainersList(); + increasedContainers = new ArrayList<>(); - for (ContainerResourceIncreaseProto c : list) { + for (ContainerProto c : list) { increasedContainers.add(convertFromProtoFormat(c)); } } @@ -403,10 +396,10 @@ private synchronized void initLocalDecreasedContainerList() { return; } AllocateResponseProtoOrBuilder p = viaProto ? proto : builder; - List list = p.getDecreasedContainersList(); - decreasedContainers = new ArrayList(); + List list = p.getDecreasedContainersList(); + decreasedContainers = new ArrayList<>(); - for (ContainerResourceDecreaseProto c : list) { + for (ContainerProto c : list) { decreasedContainers.add(convertFromProtoFormat(c)); } } @@ -453,70 +446,6 @@ private synchronized void initLocalNewNMTokenList() { } } - private synchronized Iterable - getIncreaseProtoIterable( - final List newContainersList) { - maybeInitBuilder(); - return new Iterable() { - @Override - public synchronized Iterator iterator() { - return new Iterator() { - - Iterator iter = newContainersList - .iterator(); - - @Override - public synchronized boolean hasNext() { - return iter.hasNext(); - } - - @Override - public synchronized ContainerResourceIncreaseProto next() { - return convertToProtoFormat(iter.next()); - } - - @Override - public synchronized void remove() { - throw new UnsupportedOperationException(); - } - }; - - } - }; - } - - private synchronized Iterable - getChangeProtoIterable( - final List newContainersList) { - maybeInitBuilder(); - return new Iterable() { - @Override - public synchronized Iterator iterator() { - return new Iterator() { - - Iterator iter = newContainersList - .iterator(); - - @Override - public synchronized boolean hasNext() { - return iter.hasNext(); - } - - @Override - public synchronized ContainerResourceDecreaseProto next() { - return convertToProtoFormat(iter.next()); - } - - @Override - public synchronized void remove() { - throw new UnsupportedOperationException(); - } - }; - - } - }; - } - private synchronized Iterable getContainerProtoIterable( final List newContainersList) { maybeInitBuilder(); @@ -654,26 +583,6 @@ private synchronized void initLocalFinishedContainerList() { completedContainersStatuses.add(convertFromProtoFormat(c)); } } - - private synchronized ContainerResourceIncrease convertFromProtoFormat( - ContainerResourceIncreaseProto p) { - return new ContainerResourceIncreasePBImpl(p); - } - - private synchronized ContainerResourceIncreaseProto convertToProtoFormat( - ContainerResourceIncrease t) { - return ((ContainerResourceIncreasePBImpl) t).getProto(); - } - - private synchronized ContainerResourceDecrease convertFromProtoFormat( - ContainerResourceDecreaseProto p) { - return new ContainerResourceDecreasePBImpl(p); - } - - private synchronized ContainerResourceDecreaseProto convertToProtoFormat( - ContainerResourceDecrease t) { - return ((ContainerResourceDecreasePBImpl) t).getProto(); - } private synchronized NodeReportPBImpl convertFromProtoFormat( NodeReportProto p) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java new file mode 100644 index 0000000..7417051 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.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.api.protocolrecords.impl.pb; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class IncreaseContainersResourceRequestPBImpl extends + IncreaseContainersResourceRequest { + IncreaseContainersResourceRequestProto proto = + IncreaseContainersResourceRequestProto.getDefaultInstance(); + IncreaseContainersResourceRequestProto.Builder builder = null; + boolean viaProto = false; + + private List containersToIncrease = null; + + public IncreaseContainersResourceRequestPBImpl() { + builder = IncreaseContainersResourceRequestProto.newBuilder(); + } + + public IncreaseContainersResourceRequestPBImpl( + IncreaseContainersResourceRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public IncreaseContainersResourceRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.containersToIncrease != null) { + addIncreaseContainersToProto(); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = IncreaseContainersResourceRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public List getContainersToIncrease() { + if (containersToIncrease != null) { + return containersToIncrease; + } + IncreaseContainersResourceRequestProtoOrBuilder p = + viaProto ? proto : builder; + List list = p.getIncreaseContainersList(); + containersToIncrease = new ArrayList<>(); + for (TokenProto c : list) { + containersToIncrease.add(convertFromProtoFormat(c)); + } + return containersToIncrease; + } + + @Override + public void setContainersToIncrease(List containersToIncrease) { + maybeInitBuilder(); + if (containersToIncrease == null) { + builder.clearIncreaseContainers(); + } + this.containersToIncrease = containersToIncrease; + } + + private void addIncreaseContainersToProto() { + maybeInitBuilder(); + builder.clearIncreaseContainers(); + if (this.containersToIncrease == null) { + return; + } + Iterable iterable = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + Iterator iter = containersToIncrease.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public TokenProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + builder.addAllIncreaseContainers(iterable); + } + + private Token convertFromProtoFormat(TokenProto p) { + return new TokenPBImpl(p); + } + + private TokenProto convertToProtoFormat(Token t) { + return ((TokenPBImpl) t).getProto(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java new file mode 100644 index 0000000..15062e1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java @@ -0,0 +1,241 @@ +/** + * 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 java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.SerializedException; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class IncreaseContainersResourceResponsePBImpl extends + IncreaseContainersResourceResponse { + IncreaseContainersResourceResponseProto proto = + IncreaseContainersResourceResponseProto.getDefaultInstance(); + IncreaseContainersResourceResponseProto.Builder builder = null; + boolean viaProto = false; + private List succeededRequests = null; + private Map failedRequests = null; + + public IncreaseContainersResourceResponsePBImpl() { + builder = IncreaseContainersResourceResponseProto.newBuilder(); + } + + public IncreaseContainersResourceResponsePBImpl( + IncreaseContainersResourceResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public IncreaseContainersResourceResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.succeededRequests != null) { + addSucceededRequestsToProto(); + } + if (this.failedRequests != null) { + addFailedRequestsToProto(); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = IncreaseContainersResourceResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public List getSuccessfullyIncreasedContainers() { + initSucceededRequests(); + return this.succeededRequests; + } + + @Override + public void setSuccessfullyIncreasedContainers( + List succeededRequests) { + maybeInitBuilder(); + if (succeededRequests == null) { + builder.clearSucceededRequests(); + } + this.succeededRequests = succeededRequests; + } + + private void initSucceededRequests() { + if (this.succeededRequests != null) { + return; + } + IncreaseContainersResourceResponseProtoOrBuilder p = + viaProto ? proto : builder; + List list = p.getSucceededRequestsList(); + this.succeededRequests = new ArrayList(); + for (ContainerIdProto c : list) { + this.succeededRequests.add(convertFromProtoFormat(c)); + } + } + + private void addSucceededRequestsToProto() { + maybeInitBuilder(); + builder.clearSucceededRequests(); + if (this.succeededRequests == null) { + return; + } + Iterable iterable = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + Iterator iter = succeededRequests.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public ContainerIdProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + builder.addAllSucceededRequests(iterable); + } + + @Override + public Map getFailedRequests() { + initFailedRequests(); + return this.failedRequests; + } + + @Override + public void setFailedRequests( + Map failedRequests) { + maybeInitBuilder(); + if (failedRequests == null) { + builder.clearFailedRequests(); + } + this.failedRequests = failedRequests; + } + + private void initFailedRequests() { + if (this.failedRequests != null) { + return; + } + IncreaseContainersResourceResponseProtoOrBuilder + p = viaProto ? proto : builder; + List protoList = p.getFailedRequestsList(); + this.failedRequests = new HashMap(); + for (ContainerExceptionMapProto ce : protoList) { + this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()), + convertFromProtoFormat(ce.getException())); + } + } + + private void addFailedRequestsToProto() { + maybeInitBuilder(); + builder.clearFailedRequests(); + if (this.failedRequests == null) { + return; + } + List protoList = + new ArrayList(); + + for (Map.Entry entry : this.failedRequests + .entrySet()) { + protoList.add(ContainerExceptionMapProto.newBuilder() + .setContainerId(convertToProtoFormat(entry.getKey())) + .setException(convertToProtoFormat(entry.getValue())).build()); + } + builder.addAllFailedRequests(protoList); + } + + private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { + return new ContainerIdPBImpl(p); + } + + private ContainerIdProto convertToProtoFormat(ContainerId t) { + return ((ContainerIdPBImpl) t).getProto(); + } + + private SerializedExceptionPBImpl convertFromProtoFormat( + SerializedExceptionProto p) { + return new SerializedExceptionPBImpl(p); + } + + private SerializedExceptionProto convertToProtoFormat(SerializedException t) { + return ((SerializedExceptionPBImpl) t).getProto(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java new file mode 100644 index 0000000..f382b8c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java @@ -0,0 +1,141 @@ +/** + * 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.impl.pb; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; + + +public class ContainerResourceChangeRequestPBImpl extends + ContainerResourceChangeRequest { + ContainerResourceChangeRequestProto proto = + ContainerResourceChangeRequestProto.getDefaultInstance(); + ContainerResourceChangeRequestProto.Builder builder = null; + boolean viaProto = false; + + private ContainerId existingContainerId = null; + private Resource targetCapability = null; + + public ContainerResourceChangeRequestPBImpl() { + builder = ContainerResourceChangeRequestProto.newBuilder(); + } + + public ContainerResourceChangeRequestPBImpl( + ContainerResourceChangeRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public ContainerResourceChangeRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public ContainerId getContainerId() { + ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto + : builder; + if (this.existingContainerId != null) { + return this.existingContainerId; + } + if (p.hasContainerId()) { + this.existingContainerId = convertFromProtoFormat(p.getContainerId()); + } + return this.existingContainerId; + } + + @Override + public void setContainerId(ContainerId existingContainerId) { + maybeInitBuilder(); + if (existingContainerId == null) { + builder.clearContainerId(); + } + this.existingContainerId = existingContainerId; + } + + @Override + public Resource getCapability() { + ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto + : builder; + if (this.targetCapability != null) { + return this.targetCapability; + } + if (p.hasCapability()) { + this.targetCapability = convertFromProtoFormat(p.getCapability()); + } + return this.targetCapability; + } + + @Override + public void setCapability(Resource targetCapability) { + maybeInitBuilder(); + if (targetCapability == null) { + builder.clearCapability(); + } + this.targetCapability = targetCapability; + } + + private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { + return new ContainerIdPBImpl(p); + } + + private ContainerIdProto convertToProtoFormat(ContainerId t) { + return ((ContainerIdPBImpl) t).getProto(); + } + + private Resource convertFromProtoFormat(ResourceProto p) { + return new ResourcePBImpl(p); + } + + private ResourceProto convertToProtoFormat(Resource t) { + return ((ResourcePBImpl) t).getProto(); + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ContainerResourceChangeRequestProto.newBuilder(proto); + } + viaProto = false; + } + + private void mergeLocalToBuilder() { + if (this.existingContainerId != null) { + builder.setContainerId(convertToProtoFormat(this.existingContainerId)); + } + if (this.targetCapability != null) { + builder.setCapability(convertToProtoFormat(this.targetCapability)); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java deleted file mode 100644 index 1834132..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java +++ /dev/null @@ -1,136 +0,0 @@ -/** - * 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.impl.pb; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProtoOrBuilder; -import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; - -public class ContainerResourceDecreasePBImpl extends ContainerResourceDecrease { - ContainerResourceDecreaseProto proto = ContainerResourceDecreaseProto - .getDefaultInstance(); - ContainerResourceDecreaseProto.Builder builder = null; - boolean viaProto = false; - - private ContainerId existingContainerId = null; - private Resource targetCapability = null; - - public ContainerResourceDecreasePBImpl() { - builder = ContainerResourceDecreaseProto.newBuilder(); - } - - public ContainerResourceDecreasePBImpl(ContainerResourceDecreaseProto proto) { - this.proto = proto; - viaProto = true; - } - - public ContainerResourceDecreaseProto getProto() { - mergeLocalToProto(); - proto = viaProto ? proto : builder.build(); - viaProto = true; - return proto; - } - - @Override - public ContainerId getContainerId() { - ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder; - if (this.existingContainerId != null) { - return this.existingContainerId; - } - if (p.hasContainerId()) { - this.existingContainerId = convertFromProtoFormat(p.getContainerId()); - } - return this.existingContainerId; - } - - @Override - public void setContainerId(ContainerId existingContainerId) { - maybeInitBuilder(); - if (existingContainerId == null) { - builder.clearContainerId(); - } - this.existingContainerId = existingContainerId; - } - - @Override - public Resource getCapability() { - ContainerResourceDecreaseProtoOrBuilder p = viaProto ? proto : builder; - if (this.targetCapability != null) { - return this.targetCapability; - } - if (p.hasCapability()) { - this.targetCapability = convertFromProtoFormat(p.getCapability()); - } - return this.targetCapability; - } - - @Override - public void setCapability(Resource targetCapability) { - maybeInitBuilder(); - if (targetCapability == null) { - builder.clearCapability(); - } - this.targetCapability = targetCapability; - } - - private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { - return new ContainerIdPBImpl(p); - } - - private ContainerIdProto convertToProtoFormat(ContainerId t) { - return ((ContainerIdPBImpl) t).getProto(); - } - - private Resource convertFromProtoFormat(ResourceProto p) { - return new ResourcePBImpl(p); - } - - private ResourceProto convertToProtoFormat(Resource t) { - return ((ResourcePBImpl) t).getProto(); - } - - private void mergeLocalToProto() { - if (viaProto) { - maybeInitBuilder(); - } - mergeLocalToBuilder(); - proto = builder.build(); - viaProto = true; - } - - private void maybeInitBuilder() { - if (viaProto || builder == null) { - builder = ContainerResourceDecreaseProto.newBuilder(proto); - } - viaProto = false; - } - - private void mergeLocalToBuilder() { - if (this.existingContainerId != null) { - builder.setContainerId(convertToProtoFormat(this.existingContainerId)); - } - if (this.targetCapability != null) { - builder.setCapability(convertToProtoFormat(this.targetCapability)); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java deleted file mode 100644 index 4e4f3a7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java +++ /dev/null @@ -1,171 +0,0 @@ -/** - * 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.impl.pb; - -import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProtoOrBuilder; -import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; - -public class ContainerResourceIncreasePBImpl extends ContainerResourceIncrease { - ContainerResourceIncreaseProto proto = ContainerResourceIncreaseProto - .getDefaultInstance(); - ContainerResourceIncreaseProto.Builder builder = null; - boolean viaProto = false; - - private ContainerId existingContainerId = null; - private Resource targetCapability = null; - private Token token = null; - - public ContainerResourceIncreasePBImpl() { - builder = ContainerResourceIncreaseProto.newBuilder(); - } - - public ContainerResourceIncreasePBImpl(ContainerResourceIncreaseProto proto) { - this.proto = proto; - viaProto = true; - } - - public ContainerResourceIncreaseProto getProto() { - mergeLocalToProto(); - proto = viaProto ? proto : builder.build(); - viaProto = true; - return proto; - } - - @Override - public ContainerId getContainerId() { - ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder; - if (this.existingContainerId != null) { - return this.existingContainerId; - } - if (p.hasContainerId()) { - this.existingContainerId = convertFromProtoFormat(p.getContainerId()); - } - return this.existingContainerId; - } - - @Override - public void setContainerId(ContainerId existingContainerId) { - maybeInitBuilder(); - if (existingContainerId == null) { - builder.clearContainerId(); - } - this.existingContainerId = existingContainerId; - } - - @Override - public Resource getCapability() { - ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder; - if (this.targetCapability != null) { - return this.targetCapability; - } - if (p.hasCapability()) { - this.targetCapability = convertFromProtoFormat(p.getCapability()); - } - return this.targetCapability; - } - - @Override - public void setCapability(Resource targetCapability) { - maybeInitBuilder(); - if (targetCapability == null) { - builder.clearCapability(); - } - this.targetCapability = targetCapability; - } - - @Override - public Token getContainerToken() { - ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder; - if (this.token != null) { - return this.token; - } - if (p.hasContainerToken()) { - this.token = convertFromProtoFormat(p.getContainerToken()); - } - return this.token; - } - - @Override - public void setContainerToken(Token token) { - maybeInitBuilder(); - if (token == null) { - builder.clearContainerToken(); - } - this.token = token; - } - - private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { - return new ContainerIdPBImpl(p); - } - - private ContainerIdProto convertToProtoFormat(ContainerId t) { - return ((ContainerIdPBImpl) t).getProto(); - } - - private Resource convertFromProtoFormat(ResourceProto p) { - return new ResourcePBImpl(p); - } - - private ResourceProto convertToProtoFormat(Resource t) { - return ((ResourcePBImpl) t).getProto(); - } - - private Token convertFromProtoFormat(TokenProto p) { - return new TokenPBImpl(p); - } - - private TokenProto convertToProtoFormat(Token t) { - return ((TokenPBImpl) t).getProto(); - } - - private void mergeLocalToProto() { - if (viaProto) { - maybeInitBuilder(); - } - mergeLocalToBuilder(); - proto = builder.build(); - viaProto = true; - } - - private void maybeInitBuilder() { - if (viaProto || builder == null) { - builder = ContainerResourceIncreaseProto.newBuilder(proto); - } - viaProto = false; - } - - private void mergeLocalToBuilder() { - if (this.existingContainerId != null) { - builder.setContainerId(convertToProtoFormat(this.existingContainerId)); - } - if (this.targetCapability != null) { - builder.setCapability(convertToProtoFormat(this.targetCapability)); - } - if (this.token != null) { - builder.setContainerToken(convertToProtoFormat(this.token)); - } - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java deleted file mode 100644 index f5ebf6c..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * 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.impl.pb; - -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProtoOrBuilder; -import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; - - -public class ContainerResourceIncreaseRequestPBImpl extends - ContainerResourceIncreaseRequest { - ContainerResourceIncreaseRequestProto proto = - ContainerResourceIncreaseRequestProto.getDefaultInstance(); - ContainerResourceIncreaseRequestProto.Builder builder = null; - boolean viaProto = false; - - private ContainerId existingContainerId = null; - private Resource targetCapability = null; - - public ContainerResourceIncreaseRequestPBImpl() { - builder = ContainerResourceIncreaseRequestProto.newBuilder(); - } - - public ContainerResourceIncreaseRequestPBImpl( - ContainerResourceIncreaseRequestProto proto) { - this.proto = proto; - viaProto = true; - } - - public ContainerResourceIncreaseRequestProto getProto() { - mergeLocalToProto(); - proto = viaProto ? proto : builder.build(); - viaProto = true; - return proto; - } - - @Override - public ContainerId getContainerId() { - ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto - : builder; - if (this.existingContainerId != null) { - return this.existingContainerId; - } - if (p.hasContainerId()) { - this.existingContainerId = convertFromProtoFormat(p.getContainerId()); - } - return this.existingContainerId; - } - - @Override - public void setContainerId(ContainerId existingContainerId) { - maybeInitBuilder(); - if (existingContainerId == null) { - builder.clearContainerId(); - } - this.existingContainerId = existingContainerId; - } - - @Override - public Resource getCapability() { - ContainerResourceIncreaseRequestProtoOrBuilder p = viaProto ? proto - : builder; - if (this.targetCapability != null) { - return this.targetCapability; - } - if (p.hasCapability()) { - this.targetCapability = convertFromProtoFormat(p.getCapability()); - } - return this.targetCapability; - } - - @Override - public void setCapability(Resource targetCapability) { - maybeInitBuilder(); - if (targetCapability == null) { - builder.clearCapability(); - } - this.targetCapability = targetCapability; - } - - private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { - return new ContainerIdPBImpl(p); - } - - private ContainerIdProto convertToProtoFormat(ContainerId t) { - return ((ContainerIdPBImpl) t).getProto(); - } - - private Resource convertFromProtoFormat(ResourceProto p) { - return new ResourcePBImpl(p); - } - - private ResourceProto convertToProtoFormat(Resource t) { - return ((ResourcePBImpl) t).getProto(); - } - - private void mergeLocalToProto() { - if (viaProto) { - maybeInitBuilder(); - } - mergeLocalToBuilder(); - proto = builder.build(); - viaProto = true; - } - - private void maybeInitBuilder() { - if (viaProto || builder == null) { - builder = ContainerResourceIncreaseRequestProto.newBuilder(proto); - } - viaProto = false; - } - - private void mergeLocalToBuilder() { - if (this.existingContainerId != null) { - builder.setContainerId(convertToProtoFormat(this.existingContainerId)); - } - if (this.targetCapability != null) { - builder.setCapability(convertToProtoFormat(this.targetCapability)); - } - } -} 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 86f2af9..d33d06d 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,6 +24,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; @@ -78,6 +80,7 @@ public String toString() { sb.append("ContainerStatus: ["); sb.append("ContainerId: ").append(getContainerId()).append(", "); sb.append("State: ").append(getState()).append(", "); + sb.append("Capability: ").append(getCapability()).append(", "); sb.append("Diagnostics: ").append(getDiagnostics()).append(", "); sb.append("ExitStatus: ").append(getExitStatus()).append(", "); sb.append("]"); @@ -168,6 +171,25 @@ public synchronized void setDiagnostics(String diagnostics) { builder.setDiagnostics(diagnostics); } + @Override + public synchronized Resource getCapability() { + ContainerStatusProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasCapability()) { + return null; + } + return convertFromProtoFormat(p.getCapability()); + } + + @Override + public synchronized void setCapability(Resource capability) { + maybeInitBuilder(); + if (capability == null) { + builder.clearCapability(); + return; + } + builder.setCapability(convertToProtoFormat(capability)); + } + private ContainerStateProto convertToProtoFormat(ContainerState e) { return ProtoUtils.convertToProtoFormat(e); } @@ -184,6 +206,11 @@ private ContainerIdProto convertToProtoFormat(ContainerId t) { return ((ContainerIdPBImpl)t).getProto(); } + private ResourceProto convertToProtoFormat(Resource e) { + return ((ResourcePBImpl)e).getProto(); + } - -} + private ResourcePBImpl convertFromProtoFormat(ResourceProto p) { + return new ResourcePBImpl(p); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java index c2fc1f0..2fdf214 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java @@ -110,4 +110,9 @@ public Resource multiplyAndNormalizeDown(Resource r, double by, ); } + @Override + public boolean fitsIn(Resource cluster, + Resource smaller, Resource bigger) { + return smaller.getMemory() <= bigger.getMemory(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java index 2ee95ce..b5c9967 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java @@ -209,4 +209,10 @@ public Resource multiplyAndNormalizeDown(Resource r, double by, ); } + @Override + public boolean fitsIn(Resource cluster, + Resource smaller, Resource bigger) { + return smaller.getMemory() <= bigger.getMemory() + && smaller.getVirtualCores() <= bigger.getVirtualCores(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java index 442196c..3a31225 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java @@ -171,4 +171,9 @@ public abstract float divide( */ public abstract Resource divideAndCeil(Resource numerator, int denominator); + /** + * Check if a smaller resource can be contained by bigger resource. + */ + public abstract boolean fitsIn(Resource cluster, + Resource smaller, Resource bigger); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java index 503d456..b05d021 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java @@ -267,6 +267,11 @@ public static boolean fitsIn(Resource smaller, Resource bigger) { return smaller.getMemory() <= bigger.getMemory() && smaller.getVirtualCores() <= bigger.getVirtualCores(); } + + public static boolean fitsIn(ResourceCalculator rc, Resource cluster, + Resource smaller, Resource bigger) { + return rc.fitsIn(cluster, smaller, bigger); + } public static Resource componentwiseMin(Resource lhs, Resource rhs) { return createResource(Math.min(lhs.getMemory(), rhs.getMemory()), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java index e2071dd..0a19783 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java @@ -31,6 +31,8 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -166,5 +168,11 @@ public GetContainerStatusesResponse getContainerStatuses( GetContainerStatusesResponse.newInstance(list, null); return null; } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, IOException { + return null; + } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java new file mode 100644 index 0000000..50ff1e0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java @@ -0,0 +1,162 @@ +/** + * 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; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse; +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.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC; +import org.apache.hadoop.yarn.ipc.YarnRPC; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketTimeoutException; +import java.util.ArrayList; +import java.util.List; + +/* + * Test that the container resource increase rpc times out properly. + * This is used by AM to increase container resource. + */ +public class TestContainerResourceIncreaseRPC { + + static final Log LOG = LogFactory.getLog( + TestContainerResourceIncreaseRPC.class); + + @Test + public void testHadoopProtoRPCTimeout() throws Exception { + testRPCTimeout(HadoopYarnProtoRPC.class.getName()); + } + + private void testRPCTimeout(String rpcClass) throws Exception { + Configuration conf = new Configuration(); + // set timeout low for the test + conf.setInt("yarn.rpc.nm-command-timeout", 3000); + conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass); + YarnRPC rpc = YarnRPC.create(conf); + String bindAddr = "localhost:0"; + InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr); + Server server = rpc.getServer(ContainerManagementProtocol.class, + new DummyContainerManager(), addr, conf, null, 1); + server.start(); + try { + ContainerManagementProtocol proxy = + (ContainerManagementProtocol) rpc.getProxy( + ContainerManagementProtocol.class, + server.getListenerAddress(), conf); + ApplicationId applicationId = ApplicationId.newInstance(0, 0); + ApplicationAttemptId applicationAttemptId = + ApplicationAttemptId.newInstance(applicationId, 0); + ContainerId containerId = + ContainerId.newContainerId(applicationAttemptId, 100); + NodeId nodeId = NodeId.newInstance("localhost", 1234); + Resource resource = Resource.newInstance(1234, 2); + ContainerTokenIdentifier containerTokenIdentifier = + new ContainerTokenIdentifier(containerId, "localhost", "user", + resource, System.currentTimeMillis() + 10000, 42, 42, + Priority.newInstance(0), 0); + Token containerToken = + TestRPC.newContainerToken(nodeId, "password".getBytes(), + containerTokenIdentifier); + // Construct container resource increase request, + List increaseTokens = new ArrayList<>(); + increaseTokens.add(containerToken); + IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest + .newInstance(increaseTokens); + try { + proxy.increaseContainersResource(increaseRequest); + } catch (Exception e) { + LOG.info(StringUtils.stringifyException(e)); + Assert.assertEquals("Error, exception is not: " + + SocketTimeoutException.class.getName(), + SocketTimeoutException.class.getName(), e.getClass().getName()); + return; + } + } finally { + server.stop(); + } + Assert.fail("timeout exception should have occurred!"); + } + + public class DummyContainerManager implements ContainerManagementProtocol { + + @Override + public StartContainersResponse startContainers( + StartContainersRequest requests) throws YarnException, IOException { + Exception e = new Exception("Dummy function", new Exception( + "Dummy function cause")); + throw new YarnException(e); + } + + @Override + public StopContainersResponse + stopContainers(StopContainersRequest requests) throws YarnException, + IOException { + Exception e = new Exception("Dummy function", new Exception( + "Dummy function cause")); + throw new YarnException(e); + } + + @Override + public GetContainerStatusesResponse getContainerStatuses( + GetContainerStatusesRequest request) throws YarnException, IOException { + Exception e = new Exception("Dummy function", new Exception( + "Dummy function cause")); + throw new YarnException(e); + } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, IOException { + try { + // make the thread sleep to look like its not going to respond + Thread.sleep(10000); + } catch (Exception e) { + LOG.error(e); + throw new YarnException(e); + } + throw new YarnException("Shouldn't happen!!"); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java index 39e6162..e718661 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java @@ -33,6 +33,8 @@ import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; @@ -219,6 +221,12 @@ public StopContainersResponse stopContainers(StopContainersRequest request) new Exception(EXCEPTION_CAUSE)); throw new YarnException(e); } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) throws YarnException, IOException { + return null; + } } public static ContainerTokenIdentifier newContainerTokenIdentifier( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java deleted file mode 100644 index 5ea29f8..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * 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 java.util.ArrayList; -import java.util.List; - -import org.junit.Assert; - -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto; -import org.junit.Test; - -public class TestAllocateRequest { - @Test - public void testAllcoateRequestWithIncrease() { - List incRequests = - new ArrayList(); - for (int i = 0; i < 3; i++) { - incRequests.add(ContainerResourceIncreaseRequest.newInstance(null, - Resource.newInstance(0, i))); - } - AllocateRequest r = - AllocateRequest.newInstance(123, 0f, null, null, null, incRequests); - - // serde - AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto(); - r = new AllocateRequestPBImpl(p); - - // check value - Assert.assertEquals(123, r.getResponseId()); - Assert.assertEquals(incRequests.size(), r.getIncreaseRequests().size()); - - for (int i = 0; i < incRequests.size(); i++) { - Assert.assertEquals(r.getIncreaseRequests().get(i).getCapability() - .getVirtualCores(), incRequests.get(i).getCapability() - .getVirtualCores()); - } - } - - @Test - public void testAllcoateRequestWithoutIncrease() { - AllocateRequest r = - AllocateRequest.newInstance(123, 0f, null, null, null, null); - - // serde - AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto(); - r = new AllocateRequestPBImpl(p); - - // check value - Assert.assertEquals(123, r.getResponseId()); - Assert.assertEquals(0, r.getIncreaseRequests().size()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java deleted file mode 100644 index fbe9af9..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * 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 java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; -import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl; -import org.apache.hadoop.yarn.api.records.AMCommand; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.NMToken; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto; -import org.junit.Assert; -import org.junit.Test; - -/** - * 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. - */ -public class TestAllocateResponse { - @SuppressWarnings("deprecation") - @Test - public void testAllocateResponseWithIncDecContainers() { - List incContainers = - new ArrayList(); - List decContainers = - new ArrayList(); - for (int i = 0; i < 3; i++) { - incContainers.add(ContainerResourceIncrease.newInstance(null, - Resource.newInstance(1024, i), null)); - } - for (int i = 0; i < 5; i++) { - decContainers.add(ContainerResourceDecrease.newInstance(null, - Resource.newInstance(1024, i))); - } - - AllocateResponse r = - AllocateResponse.newInstance(3, new ArrayList(), - new ArrayList(), new ArrayList(), null, - AMCommand.AM_RESYNC, 3, null, new ArrayList(), - incContainers, decContainers); - - // serde - AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto(); - r = new AllocateResponsePBImpl(p); - - // check value - Assert - .assertEquals(incContainers.size(), r.getIncreasedContainers().size()); - Assert - .assertEquals(decContainers.size(), r.getDecreasedContainers().size()); - - for (int i = 0; i < incContainers.size(); i++) { - Assert.assertEquals(i, r.getIncreasedContainers().get(i).getCapability() - .getVirtualCores()); - } - - for (int i = 0; i < decContainers.size(); i++) { - Assert.assertEquals(i, r.getDecreasedContainers().get(i).getCapability() - .getVirtualCores()); - } - } - - @SuppressWarnings("deprecation") - @Test - public void testAllocateResponseWithoutIncDecContainers() { - AllocateResponse r = - AllocateResponse.newInstance(3, new ArrayList(), - new ArrayList(), new ArrayList(), null, - AMCommand.AM_RESYNC, 3, null, new ArrayList(), null, null); - - // serde - AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto(); - r = new AllocateResponsePBImpl(p); - - // check value - Assert.assertEquals(0, r.getIncreasedContainers().size()); - Assert.assertEquals(0, r.getDecreasedContainers().size()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java deleted file mode 100644 index 29b0ffe..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * 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.junit.Assert; - -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.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto; -import org.junit.Test; - -public class TestContainerResourceDecrease { - @Test - public void testResourceDecreaseContext() { - ContainerId containerId = ContainerId - .newContainerId(ApplicationAttemptId.newInstance( - ApplicationId.newInstance(1234, 3), 3), 7); - Resource resource = Resource.newInstance(1023, 3); - ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance( - containerId, resource); - - // get proto and recover to ctx - ContainerResourceDecreaseProto proto = - ((ContainerResourceDecreasePBImpl) ctx).getProto(); - ctx = new ContainerResourceDecreasePBImpl(proto); - - // check values - Assert.assertEquals(ctx.getCapability(), resource); - Assert.assertEquals(ctx.getContainerId(), containerId); - } - - @Test - public void testResourceDecreaseContextWithNull() { - ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(null, - null); - - // get proto and recover to ctx; - ContainerResourceDecreaseProto proto = - ((ContainerResourceDecreasePBImpl) ctx).getProto(); - ctx = new ContainerResourceDecreasePBImpl(proto); - - // check values - Assert.assertNull(ctx.getCapability()); - Assert.assertNull(ctx.getContainerId()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java deleted file mode 100644 index 932d5a7..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * 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 java.util.Arrays; - -import org.junit.Assert; - -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.ContainerResourceIncrease; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.Token; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto; -import org.junit.Test; - -public class TestContainerResourceIncrease { - @Test - public void testResourceIncreaseContext() { - byte[] identifier = new byte[] { 1, 2, 3, 4 }; - Token token = Token.newInstance(identifier, "", "".getBytes(), ""); - ContainerId containerId = ContainerId - .newContainerId(ApplicationAttemptId.newInstance( - ApplicationId.newInstance(1234, 3), 3), 7); - Resource resource = Resource.newInstance(1023, 3); - ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance( - containerId, resource, token); - - // get proto and recover to ctx - ContainerResourceIncreaseProto proto = - ((ContainerResourceIncreasePBImpl) ctx).getProto(); - ctx = new ContainerResourceIncreasePBImpl(proto); - - // check values - Assert.assertEquals(ctx.getCapability(), resource); - Assert.assertEquals(ctx.getContainerId(), containerId); - Assert.assertTrue(Arrays.equals(ctx.getContainerToken().getIdentifier() - .array(), identifier)); - } - - @Test - public void testResourceIncreaseContextWithNull() { - ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(null, - null, null); - - // get proto and recover to ctx; - ContainerResourceIncreaseProto proto = - ((ContainerResourceIncreasePBImpl) ctx).getProto(); - ctx = new ContainerResourceIncreasePBImpl(proto); - - // check values - Assert.assertNull(ctx.getContainerToken()); - Assert.assertNull(ctx.getCapability()); - Assert.assertNull(ctx.getContainerId()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java deleted file mode 100644 index cf4dabf..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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.junit.Assert; - -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.ContainerResourceIncreaseRequest; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto; -import org.junit.Test; - -public class TestContainerResourceIncreaseRequest { - @Test - public void ContainerResourceIncreaseRequest() { - ContainerId containerId = ContainerId - .newContainerId(ApplicationAttemptId.newInstance( - ApplicationId.newInstance(1234, 3), 3), 7); - Resource resource = Resource.newInstance(1023, 3); - ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest - .newInstance(containerId, resource); - - // to proto and get it back - ContainerResourceIncreaseRequestProto proto = - ((ContainerResourceIncreaseRequestPBImpl) context).getProto(); - ContainerResourceIncreaseRequest contextRecover = - new ContainerResourceIncreaseRequestPBImpl(proto); - - // check value - Assert.assertEquals(contextRecover.getContainerId(), containerId); - Assert.assertEquals(contextRecover.getCapability(), resource); - } - - @Test - public void testResourceChangeContextWithNullField() { - ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest - .newInstance(null, null); - - // to proto and get it back - ContainerResourceIncreaseRequestProto proto = - ((ContainerResourceIncreaseRequestPBImpl) context).getProto(); - ContainerResourceIncreaseRequest contextRecover = - new ContainerResourceIncreaseRequestPBImpl(proto); - - // check value - Assert.assertNull(contextRecover.getContainerId()); - Assert.assertNull(contextRecover.getCapability()); - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java index 6357c36..5f707b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java @@ -44,6 +44,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; 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.CancelDelegationTokenRequestPBImpl; @@ -101,6 +103,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -113,9 +117,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerReport; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LogAggregationContext; @@ -155,9 +157,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl; @@ -190,9 +190,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto; -import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; @@ -284,6 +282,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto; @@ -471,9 +471,9 @@ public static void setup() throws Exception { generateByNewInstance(ContainerLaunchContext.class); generateByNewInstance(ApplicationSubmissionContext.class); generateByNewInstance(ContainerReport.class); - generateByNewInstance(ContainerResourceDecrease.class); - generateByNewInstance(ContainerResourceIncrease.class); - generateByNewInstance(ContainerResourceIncreaseRequest.class); + generateByNewInstance(ContainerResourceChangeRequest.class); + generateByNewInstance(IncreaseContainersResourceRequest.class); + generateByNewInstance(IncreaseContainersResourceResponse.class); generateByNewInstance(ContainerStatus.class); generateByNewInstance(PreemptionContainer.class); generateByNewInstance(PreemptionResourceRequest.class); @@ -879,6 +879,18 @@ public void testStopContainersResponsePBImpl() throws Exception { } @Test + public void testIncreaseContainersResourceRequestPBImpl() throws Exception { + validatePBImplRecord(IncreaseContainersResourceRequestPBImpl.class, + IncreaseContainersResourceRequestProto.class); + } + + @Test + public void testIncreaseContainersResourceResponsePBImpl() throws Exception { + validatePBImplRecord(IncreaseContainersResourceResponsePBImpl.class, + IncreaseContainersResourceResponseProto.class); + } + + @Test public void testSubmitApplicationRequestPBImpl() throws Exception { validatePBImplRecord(SubmitApplicationRequestPBImpl.class, SubmitApplicationRequestProto.class); @@ -959,21 +971,9 @@ public void testContainerReportPBImpl() throws Exception { } @Test - public void testContainerResourceDecreasePBImpl() throws Exception { - validatePBImplRecord(ContainerResourceDecreasePBImpl.class, - ContainerResourceDecreaseProto.class); - } - - @Test - public void testContainerResourceIncreasePBImpl() throws Exception { - validatePBImplRecord(ContainerResourceIncreasePBImpl.class, - ContainerResourceIncreaseProto.class); - } - - @Test - public void testContainerResourceIncreaseRequestPBImpl() throws Exception { - validatePBImplRecord(ContainerResourceIncreaseRequestPBImpl.class, - ContainerResourceIncreaseRequestProto.class); + public void testContainerResourceChangeRequestPBImpl() throws Exception { + validatePBImplRecord(ContainerResourceChangeRequestPBImpl.class, + ContainerResourceChangeRequestProto.class); } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java index 6a0b62e..0654891 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java @@ -41,6 +41,35 @@ public TestResourceCalculator(ResourceCalculator rs) { this.resourceCalculator = rs; } + + @Test(timeout = 10000) + public void testFitsIn() { + Resource cluster = Resource.newInstance(1024, 1); + + if (resourceCalculator instanceof DefaultResourceCalculator) { + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(2, 1))); + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(2, 2))); + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(1, 2))); + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(1, 1))); + Assert.assertFalse(resourceCalculator.fitsIn(cluster, + Resource.newInstance(2, 1), Resource.newInstance(1, 2))); + } else if (resourceCalculator instanceof DominantResourceCalculator) { + Assert.assertFalse(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(2, 1))); + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(2, 2))); + Assert.assertTrue(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(1, 2))); + Assert.assertFalse(resourceCalculator.fitsIn(cluster, + Resource.newInstance(1, 2), Resource.newInstance(1, 1))); + Assert.assertFalse(resourceCalculator.fitsIn(cluster, + Resource.newInstance(2, 1), Resource.newInstance(1, 2))); + } + } @Test(timeout = 10000) public void testResourceCalculatorCompareMethod() { @@ -92,7 +121,6 @@ public void testResourceCalculatorCompareMethod() { } - private void assertResourcesOperations(Resource clusterResource, Resource lhs, Resource rhs, boolean lessThan, boolean lessThanOrEqual, boolean greaterThan, boolean greaterThanOrEqual, Resource max, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java index 1498a0c..c0ccf57 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java @@ -19,10 +19,12 @@ package org.apache.hadoop.yarn.server.api.protocolrecords; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.List; import java.util.Map; 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.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.NodeAction; @@ -70,4 +72,7 @@ void setSystemCredentialsForApps( boolean getAreNodeLabelsAcceptedByRM(); void setAreNodeLabelsAcceptedByRM(boolean areNodeLabelsAcceptedByRM); + + List getContainersToDecrease(); + void addAllContainersToDecrease(Collection containersToDecrease); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java index e27d8ca..dc65141 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java @@ -20,19 +20,23 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; 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.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase; import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto; import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto; @@ -58,7 +62,9 @@ private MasterKey containerTokenMasterKey = null; private MasterKey nmTokenMasterKey = null; - + + private List containersToDecrease = null; + public NodeHeartbeatResponsePBImpl() { builder = NodeHeartbeatResponseProto.newBuilder(); } @@ -96,6 +102,9 @@ private void mergeLocalToBuilder() { if (this.systemCredentials != null) { addSystemCredentialsToProto(); } + if (this.containersToDecrease != null) { + addContainersToDecreaseToProto(); + } } private void addSystemCredentialsToProto() { @@ -408,6 +417,64 @@ public void remove() { builder.addAllApplicationsToCleanup(iterable); } + private void initContainersToDecrease() { + if (this.containersToDecrease != null) { + return; + } + NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getContainersToDecreaseList(); + this.containersToDecrease = new ArrayList<>(); + + for (ContainerProto c : list) { + this.containersToDecrease.add(convertFromProtoFormat(c)); + } + } + + @Override + public List getContainersToDecrease() { + initContainersToDecrease(); + return this.containersToDecrease; + } + + @Override + public void addAllContainersToDecrease( + final Collection containersToDecrease) { + if (containersToDecrease == null) { + return; + } + initContainersToDecrease(); + this.containersToDecrease.addAll(containersToDecrease); + } + + private void addContainersToDecreaseToProto() { + maybeInitBuilder(); + builder.clearContainersToDecrease(); + if (this.containersToDecrease == null) { + return; + } + Iterable iterable = new + Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + private Iterator iter = containersToDecrease.iterator(); + @Override + public boolean hasNext() { + return iter.hasNext(); + } + @Override + public ContainerProto next() { + return convertToProtoFormat(iter.next()); + } + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + builder.addAllContainersToDecrease(iterable); + } @Override public Map getSystemCredentialsForApps() { @@ -484,6 +551,14 @@ private MasterKeyProto convertToProtoFormat(MasterKey t) { return ((MasterKeyPBImpl) t).getProto(); } + private ContainerPBImpl convertFromProtoFormat(ContainerProto p) { + return new ContainerPBImpl(p); + } + + private ContainerProto convertToProtoFormat(Container t) { + return ((ContainerPBImpl) t).getProto(); + } + @Override public boolean getAreNodeLabelsAcceptedByRM() { NodeHeartbeatResponseProtoOrBuilder p = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java index 7b8262f..2d62db5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.Records; @@ -48,6 +49,7 @@ * @param nodeHealthStatus Health status of the node. * @param containersUtilization Utilization of the containers in this node. * @param nodeUtilization Utilization of the node. + * @param increasedContainers Containers whose resource has been increased. * @return New {@code NodeStatus} with the provided information. */ public static NodeStatus newInstance(NodeId nodeId, int responseId, @@ -55,7 +57,8 @@ public static NodeStatus newInstance(NodeId nodeId, int responseId, List keepAliveApplications, NodeHealthStatus nodeHealthStatus, ResourceUtilization containersUtilization, - ResourceUtilization nodeUtilization) { + ResourceUtilization nodeUtilization, + List increasedContainers) { NodeStatus nodeStatus = Records.newRecord(NodeStatus.class); nodeStatus.setResponseId(responseId); nodeStatus.setNodeId(nodeId); @@ -64,6 +67,7 @@ public static NodeStatus newInstance(NodeId nodeId, int responseId, nodeStatus.setNodeHealthStatus(nodeHealthStatus); nodeStatus.setContainersUtilization(containersUtilization); nodeStatus.setNodeUtilization(nodeUtilization); + nodeStatus.setIncreasedContainers(increasedContainers); return nodeStatus; } @@ -108,4 +112,13 @@ public abstract void setContainersUtilization( @Unstable public abstract void setNodeUtilization( ResourceUtilization nodeUtilization); + + @Public + @Unstable + public abstract List getIncreasedContainers(); + + @Private + @Unstable + public abstract void setIncreasedContainers( + List increasedContainers); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java index 7d4e83f..e34451d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java @@ -24,13 +24,16 @@ import java.util.List; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto; @@ -49,7 +52,8 @@ private List containers = null; private NodeHealthStatus nodeHealthStatus = null; private List keepAliveApplications = null; - + private List increasedContainers = null; + public NodeStatusPBImpl() { builder = NodeStatusProto.newBuilder(); } @@ -79,6 +83,9 @@ private synchronized void mergeLocalToBuilder() { if (this.keepAliveApplications != null) { addKeepAliveApplicationsToProto(); } + if (this.increasedContainers != null) { + addIncreasedContainersToProto(); + } } private synchronized void mergeLocalToProto() { @@ -165,6 +172,37 @@ public void remove() { builder.addAllKeepAliveApplications(iterable); } + private synchronized void addIncreasedContainersToProto() { + maybeInitBuilder(); + builder.clearIncreasedContainers(); + if (increasedContainers == null) { + return; + } + Iterable iterable = new + Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + private Iterator iter = + increasedContainers.iterator(); + @Override + public boolean hasNext() { + return iter.hasNext(); + } + @Override + public ContainerProto next() { + return convertToProtoFormat(iter.next()); + } + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + builder.addAllIncreasedContainers(iterable); + } + @Override public int hashCode() { return getProto().hashCode(); @@ -336,6 +374,31 @@ public synchronized void setNodeUtilization( .setNodeUtilization(convertToProtoFormat(nodeUtilization)); } + @Override + public synchronized List getIncreasedContainers() { + if (increasedContainers != null) { + return increasedContainers; + } + NodeStatusProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getIncreasedContainersList(); + this.increasedContainers = new ArrayList<>(); + for (ContainerProto c : list) { + this.increasedContainers.add(convertFromProtoFormat(c)); + } + return this.increasedContainers; + } + + @Override + public synchronized void setIncreasedContainers( + List increasedContainers) { + maybeInitBuilder(); + if (increasedContainers == null) { + builder.clearIncreasedContainers(); + return; + } + this.increasedContainers = increasedContainers; + } + private NodeIdProto convertToProtoFormat(NodeId nodeId) { return ((NodeIdPBImpl)nodeId).getProto(); } @@ -377,4 +440,14 @@ private ResourceUtilizationPBImpl convertFromProtoFormat( ResourceUtilizationProto p) { return new ResourceUtilizationPBImpl(p); } + + private ContainerPBImpl convertFromProtoFormat( + ContainerProto c) { + return new ContainerPBImpl(c); + } + + private ContainerProto convertToProtoFormat( + Container c) { + return ((ContainerPBImpl)c).getProto(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java index a3bd6f8..475e9fe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java @@ -200,13 +200,15 @@ public static NodeReport newNodeReport(NodeId nodeId, NodeState nodeState, } public static ContainerStatus newContainerStatus(ContainerId containerId, - ContainerState containerState, String diagnostics, int exitStatus) { + ContainerState containerState, String diagnostics, int exitStatus, + Resource capability) { ContainerStatus containerStatus = recordFactory .newRecordInstance(ContainerStatus.class); containerStatus.setState(containerState); containerStatus.setContainerId(containerId); containerStatus.setDiagnostics(diagnostics); containerStatus.setExitStatus(exitStatus); + containerStatus.setCapability(capability); return containerStatus; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto index 901051f..b161f5b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto @@ -38,6 +38,7 @@ message NodeStatusProto { repeated ApplicationIdProto keep_alive_applications = 5; optional ResourceUtilizationProto containers_utilization = 6; optional ResourceUtilizationProto node_utilization = 7; + repeated ContainerProto increased_containers = 8; } message MasterKeyProto { @@ -60,4 +61,4 @@ message ResourceUtilizationProto { optional int32 pmem = 1; optional int32 vmem = 2; optional float cpu = 3; -} \ No newline at end of file +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto index c122b2a..2db8919 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto @@ -82,6 +82,7 @@ message NodeHeartbeatResponseProto { repeated ContainerIdProto containers_to_be_removed_from_nm = 9; repeated SystemCredentialsForAppsProto system_credentials_for_apps = 10; optional bool areNodeLabelsAcceptedByRM = 11 [default = false]; + repeated ContainerProto containers_to_decrease = 12; } message SystemCredentialsForAppsProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java index d9eeb9d..c9427dd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.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.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -168,6 +169,20 @@ public void testNodeHeartbeatResponsePBImplWithRMAcceptLbls() { assertTrue(copy.getAreNodeLabelsAcceptedByRM()); } + @Test + public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() { + NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl(); + original.addAllContainersToDecrease( + Arrays.asList(getDecreasedContainer(1, 2, 2048, 2), + getDecreasedContainer(2, 3, 1024, 1))); + NodeHeartbeatResponsePBImpl copy = + new NodeHeartbeatResponsePBImpl(original.getProto()); + assertEquals(1, copy.getContainersToDecrease().get(0) + .getId().getContainerId()); + assertEquals(1024, copy.getContainersToDecrease().get(1) + .getResource().getMemory()); + } + /** * Test RegisterNodeManagerRequestPBImpl. */ @@ -244,6 +259,9 @@ public void testNodeStatusPBImpl() { original.setNodeHealthStatus(getNodeHealthStatus()); original.setNodeId(getNodeId()); original.setResponseId(1); + original.setIncreasedContainers( + Arrays.asList(getIncreasedContainer(1, 2, 2048, 2), + getIncreasedContainer(2, 3, 4096, 3))); NodeStatusPBImpl copy = new NodeStatusPBImpl(original.getProto()); assertEquals(3L, copy.getContainersStatuses().get(1).getContainerId() @@ -252,7 +270,10 @@ public void testNodeStatusPBImpl() { assertEquals(1000, copy.getNodeHealthStatus().getLastHealthReportTime()); assertEquals(9090, copy.getNodeId().getPort()); assertEquals(1, copy.getResponseId()); - + assertEquals(1, copy.getIncreasedContainers().get(0) + .getId().getContainerId()); + assertEquals(4096, copy.getIncreasedContainers().get(1) + .getResource().getMemory()); } @Test @@ -347,6 +368,22 @@ public ApplicationIdPBImpl setParameters(int id, long timestamp) { return new ApplicationIdPBImpl(appId.getProto()); } + private Container getDecreasedContainer(int containerID, + int appAttemptId, int memory, int vCores) { + ContainerId containerId = getContainerId(containerID, appAttemptId); + Resource capability = Resource.newInstance(memory, vCores); + return Container.newInstance( + containerId, null, null, capability, null, null); + } + + private Container getIncreasedContainer(int containerID, + int appAttemptId, int memory, int vCores) { + ContainerId containerId = getContainerId(containerID, appAttemptId); + Resource capability = Resource.newInstance(memory, vCores); + return Container.newInstance( + containerId, null, null, capability, null, null); + } + private NodeStatus getNodeStatus() { NodeStatus status = recordFactory.newRecordInstance(NodeStatus.class); status.setContainersStatuses(new ArrayList()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java new file mode 100644 index 0000000..9479d0b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java @@ -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. + */ + +package org.apache.hadoop.yarn.server.nodemanager; + +import org.apache.hadoop.yarn.api.records.Container; +import java.util.List; + +public class CMgrDecreaseContainersResourceEvent extends ContainerManagerEvent { + + private final List containersToDecrease; + + public CMgrDecreaseContainersResourceEvent(List + containersToDecrease) { + super(ContainerManagerEventType.DECREASE_CONTAINERS_RESOURCE); + this.containersToDecrease = containersToDecrease; + } + + public List getContainersToDecrease() { + return this.containersToDecrease; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java index 4278ce0..fcb0252 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java @@ -21,4 +21,5 @@ public enum ContainerManagerEventType { FINISH_APPS, FINISH_CONTAINERS, + DECREASE_CONTAINERS_RESOURCE } 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 52d937b..9c2d1fb 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 @@ -62,6 +62,9 @@ ConcurrentMap getContainers(); + ConcurrentMap + getIncreasedContainers(); + NMContainerTokenSecretManager getContainerTokenSecretManager(); NMTokenSecretManagerInNM getNMTokenSecretManager(); 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 3cf9f1a..184f489 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 @@ -439,6 +439,10 @@ public void run() { protected final ConcurrentMap containers = new ConcurrentSkipListMap(); + protected final ConcurrentMap increasedContainers = + new ConcurrentHashMap<>(); + private final NMContainerTokenSecretManager containerTokenSecretManager; private final NMTokenSecretManagerInNM nmTokenSecretManager; private ContainerManagementProtocol containerManager; @@ -493,6 +497,12 @@ public int getHttpPort() { } @Override + public ConcurrentMap + getIncreasedContainers() { + return this.increasedContainers; + } + + @Override public NMContainerTokenSecretManager getContainerTokenSecretManager() { return this.containerTokenSecretManager; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java index aa51e5c..f8ce90f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java @@ -310,18 +310,28 @@ protected ResourceTracker getRMClient() throws IOException { @VisibleForTesting protected void registerWithRM() throws YarnException, IOException { - List containerReports = getNMContainerStatuses(); + RegisterNodeManagerResponse regNMResponse; Set nodeLabels = nodeLabelsHandler.getNodeLabelsForRegistration(); - RegisterNodeManagerRequest request = - RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource, - nodeManagerVersionId, containerReports, getRunningApplications(), - nodeLabels); - if (containerReports != null) { - LOG.info("Registering with RM using containers :" + containerReports); + + // Synchronize NM-RM registration with + // ContainerManagerImpl#increaseContainersResource and + // ContainerManagerImpl#startContainers to avoid race condition + // during RM recovery + synchronized (this.context) { + List containerReports = getNMContainerStatuses(); + RegisterNodeManagerRequest request = + RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource, + nodeManagerVersionId, containerReports, getRunningApplications(), + nodeLabels); + if (containerReports != null) { + LOG.info("Registering with RM using containers :" + containerReports); + } + regNMResponse = + resourceTracker.registerNodeManager(request); + // Make sure rmIdentifier is set before we release the lock + this.rmIdentifier = regNMResponse.getRMIdentifier(); } - RegisterNodeManagerResponse regNMResponse = - resourceTracker.registerNodeManager(request); - this.rmIdentifier = regNMResponse.getRMIdentifier(); + // if the Resource Manager instructs NM to shutdown. if (NodeAction.SHUTDOWN.equals(regNMResponse.getNodeAction())) { String message = @@ -418,10 +428,12 @@ private NodeStatus getNodeStatus(int responseId) throws IOException { List containersStatuses = getContainerStatuses(); ResourceUtilization containersUtilization = getContainersUtilization(); ResourceUtilization nodeUtilization = getNodeUtilization(); + List increasedContainers + = getIncreasedContainers(); NodeStatus nodeStatus = NodeStatus.newInstance(nodeId, responseId, containersStatuses, createKeepAliveApplicationList(), nodeHealthStatus, - containersUtilization, nodeUtilization); + containersUtilization, nodeUtilization, increasedContainers); return nodeStatus; } @@ -448,6 +460,21 @@ private ResourceUtilization getNodeUtilization() { return nodeResourceMonitor.getUtilization(); } + /* Get the containers whose resource has been increased since last + * NM-RM heartbeat. + */ + private List + getIncreasedContainers() { + List + increasedContainers = new ArrayList<>( + this.context.getIncreasedContainers().values()); + for (org.apache.hadoop.yarn.api.records.Container + container : increasedContainers) { + this.context.getIncreasedContainers().remove(container.getId()); + } + return increasedContainers; + } + // Iterate through the NMContext and clone and get all the containers' // statuses. If it's a completed container, add into the // recentlyStoppedContainers collections. @@ -765,6 +792,14 @@ public void run() { ((NMContext) context) .setSystemCrendentialsForApps(parseCredentials(systemCredentials)); } + + List + containersToDecrease = response.getContainersToDecrease(); + if (!containersToDecrease.isEmpty()) { + dispatcher.getEventHandler().handle( + new CMgrDecreaseContainersResourceEvent(containersToDecrease) + ); + } } catch (ConnectException e) { //catch and throw the exception if tried MAX wait time to connect RM dispatcher.getEventHandler().handle( 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/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java index a658e53..39d2983 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java @@ -58,6 +58,8 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse; @@ -72,6 +74,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.LogAggregationContext; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.SerializedException; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl; @@ -93,6 +96,7 @@ import org.apache.hadoop.yarn.server.api.ContainerType; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent; +import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent; import org.apache.hadoop.yarn.server.nodemanager.Context; @@ -125,6 +129,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl; @@ -139,6 +144,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; +import org.apache.hadoop.yarn.util.resource.Resources; public class ContainerManagerImpl extends CompositeService implements ServiceStateChangeListener, ContainerManagementProtocol, @@ -340,7 +346,7 @@ private void recoverContainer(RecoveredContainerState rcs) Container container = new ContainerImpl(getConfig(), dispatcher, context.getNMStateStore(), req.getContainerLaunchContext(), credentials, metrics, token, rcs.getStatus(), rcs.getExitCode(), - rcs.getDiagnostics(), rcs.getKilled()); + rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability()); context.getContainers().put(containerId, container); dispatcher.getEventHandler().handle( new ApplicationContainerInitEvent(container)); @@ -557,8 +563,7 @@ public void cleanUpApplicationsOnNMShutDown() { List appIds = new ArrayList(applications.keySet()); - this.handle( - new CMgrCompletedAppsEvent(appIds, + this.handle(new CMgrCompletedAppsEvent(appIds, CMgrCompletedAppsEvent.Reason.ON_SHUTDOWN)); LOG.info("Waiting for Applications to be Finished"); @@ -578,8 +583,8 @@ public void cleanUpApplicationsOnNMShutDown() { if (applications.isEmpty()) { LOG.info("All applications in FINISHED state"); } else { - LOG.info("Done waiting for Applications to be Finished. Still alive: " + - applications.keySet()); + LOG.info("Done waiting for Applications to be Finished. Still alive: " + + applications.keySet()); } } @@ -679,33 +684,45 @@ protected void authorizeUser(UserGroupInformation remoteUgi, /** * @param containerTokenIdentifier - * of the container to be started + * of the container whose resource is to be started or increased * @throws YarnException */ @Private @VisibleForTesting - protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier, - ContainerTokenIdentifier containerTokenIdentifier) throws YarnException { + protected void authorizeStartAndResourceIncreaseRequest( + NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + boolean startRequest) + throws YarnException { if (nmTokenIdentifier == null) { throw RPCUtil.getRemoteException(INVALID_NMTOKEN_MSG); } if (containerTokenIdentifier == null) { throw RPCUtil.getRemoteException(INVALID_CONTAINERTOKEN_MSG); } + /* + * Check the following: + * 1. The request comes from the same application attempt + * 2. The request possess a container token that has not expired + * 3. The request possess a container token that is granted by a known RM + */ ContainerId containerId = containerTokenIdentifier.getContainerID(); String containerIDStr = containerId.toString(); boolean unauthorized = false; StringBuilder messageBuilder = - new StringBuilder("Unauthorized request to start container. "); + new StringBuilder("Unauthorized request to " + (startRequest ? + "start container." : "increase container resource.")); if (!nmTokenIdentifier.getApplicationAttemptId().getApplicationId(). equals(containerId.getApplicationAttemptId().getApplicationId())) { unauthorized = true; messageBuilder.append("\nNMToken for application attempt : ") .append(nmTokenIdentifier.getApplicationAttemptId()) - .append(" was used for starting container with container token") + .append(" was used for " + + (startRequest ? "starting " : "increasing resource of ") + + "container with container token") .append(" issued for application attempt : ") .append(containerId.getApplicationAttemptId()); - } else if (!this.context.getContainerTokenSecretManager() + } else if (startRequest && !this.context.getContainerTokenSecretManager() .isValidStartContainerRequest(containerTokenIdentifier)) { // Is the container being relaunched? Or RPC layer let startCall with // tokens generated off old-secret through? @@ -727,62 +744,77 @@ protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier, LOG.error(msg); throw RPCUtil.getRemoteException(msg); } + if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater + .getRMIdentifier()) { + // Is the container coming from unknown RM + StringBuilder sb = new StringBuilder("\nContainer "); + sb.append(containerTokenIdentifier.getContainerID().toString()) + .append(" rejected as it is allocated by a previous RM"); + throw new InvalidContainerException(sb.toString()); + } } /** * Start a list of containers on this NodeManager. */ @Override - public StartContainersResponse - startContainers(StartContainersRequest requests) throws YarnException, - IOException { + public StartContainersResponse startContainers( + StartContainersRequest requests) throws YarnException, IOException { if (blockNewContainerRequests.get()) { throw new NMNotYetReadyException( - "Rejecting new containers as NodeManager has not" - + " yet connected with ResourceManager"); + "Rejecting new containers as NodeManager has not" + + " yet connected with ResourceManager"); } UserGroupInformation remoteUgi = getRemoteUgi(); NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi); - authorizeUser(remoteUgi,nmTokenIdentifier); + authorizeUser(remoteUgi, nmTokenIdentifier); List succeededContainers = new ArrayList(); Map failedContainers = new HashMap(); - for (StartContainerRequest request : requests.getStartContainerRequests()) { - ContainerId containerId = null; - try { - if (request.getContainerToken() == null || - request.getContainerToken().getIdentifier() == null) { - throw new IOException(INVALID_CONTAINERTOKEN_MSG); - } - ContainerTokenIdentifier containerTokenIdentifier = - BuilderUtils.newContainerTokenIdentifier(request.getContainerToken()); - verifyAndGetContainerTokenIdentifier(request.getContainerToken(), - containerTokenIdentifier); - containerId = containerTokenIdentifier.getContainerID(); - - // Initialize the AMRMProxy service instance only if the container is of - // type AM and if the AMRMProxy service is enabled - if (isARMRMProxyEnabled() - && containerTokenIdentifier.getContainerType().equals( - ContainerType.APPLICATION_MASTER)) { - this.amrmProxyService.processApplicationStartRequest(request); - } + // Synchronize with NodeStatusUpdaterImpl#registerWithRM + // to avoid race condition during NM-RM resync (due to RM restart) while a + // container is being started, in particular when the container has not yet + // been added to the containers map in NMContext. + synchronized (this.context) { + for (StartContainerRequest request : requests + .getStartContainerRequests()) { + ContainerId containerId = null; + try { + if (request.getContainerToken() == null + || request.getContainerToken().getIdentifier() == null) { + throw new IOException(INVALID_CONTAINERTOKEN_MSG); + } - startContainerInternal(nmTokenIdentifier, - containerTokenIdentifier, request); - succeededContainers.add(containerId); - } catch (YarnException e) { - failedContainers.put(containerId, SerializedException.newInstance(e)); - } catch (InvalidToken ie) { - failedContainers.put(containerId, SerializedException.newInstance(ie)); - throw ie; - } catch (IOException e) { - throw RPCUtil.getRemoteException(e); + ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils + .newContainerTokenIdentifier(request.getContainerToken()); + verifyAndGetContainerTokenIdentifier(request.getContainerToken(), + containerTokenIdentifier); + containerId = containerTokenIdentifier.getContainerID(); + + // Initialize the AMRMProxy service instance only if the container is of + // type AM and if the AMRMProxy service is enabled + if (isARMRMProxyEnabled() && containerTokenIdentifier + .getContainerType().equals(ContainerType.APPLICATION_MASTER)) { + this.amrmProxyService.processApplicationStartRequest(request); + } + + startContainerInternal(nmTokenIdentifier, containerTokenIdentifier, + request); + succeededContainers.add(containerId); + } catch (YarnException e) { + failedContainers.put(containerId, SerializedException.newInstance(e)); + } catch (InvalidToken ie) { + failedContainers + .put(containerId, SerializedException.newInstance(ie)); + throw ie; + } catch (IOException e) { + throw RPCUtil.getRemoteException(e); + } } + return StartContainersResponse + .newInstance(getAuxServiceMetaData(), succeededContainers, + failedContainers); } - - return StartContainersResponse.newInstance(getAuxServiceMetaData(), - succeededContainers, failedContainers); } private ContainerManagerApplicationProto buildAppProto(ApplicationId appId, @@ -842,16 +874,8 @@ private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier, * belongs to correct Node Manager (part of retrieve password). c) It has * correct RMIdentifier. d) It is not expired. */ - authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier); - - if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater - .getRMIdentifier()) { - // Is the container coming from unknown RM - StringBuilder sb = new StringBuilder("\nContainer "); - sb.append(containerTokenIdentifier.getContainerID().toString()) - .append(" rejected as it is allocated by a previous RM"); - throw new InvalidContainerException(sb.toString()); - } + authorizeStartAndResourceIncreaseRequest( + nmTokenIdentifier, containerTokenIdentifier, true); // update NMToken updateNMTokenIdentifier(nmTokenIdentifier); @@ -941,7 +965,7 @@ protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier( InvalidToken { byte[] password = context.getContainerTokenSecretManager().retrievePassword( - containerTokenIdentifier); + containerTokenIdentifier); byte[] tokenPass = token.getPassword().array(); if (password == null || tokenPass == null || !Arrays.equals(password, tokenPass)) { @@ -952,6 +976,147 @@ protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier( return containerTokenIdentifier; } + /** + * Increase resource of a list of containers on this NodeManager. + */ + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest requests) + throws YarnException, IOException { + if (blockNewContainerRequests.get()) { + throw new NMNotYetReadyException( + "Rejecting container resource increase as NodeManager has not" + + " yet connected with ResourceManager"); + } + UserGroupInformation remoteUgi = getRemoteUgi(); + NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi); + authorizeUser(remoteUgi, nmTokenIdentifier); + List successfullyIncreasedContainers + = new ArrayList(); + Map failedContainers = + new HashMap(); + // Synchronize with NodeStatusUpdaterImpl#registerWithRM + // to avoid race condition during NM-RM resync (due to RM restart) while a + // container resource is being increased in NM, in particular when the + // increased container has not yet been added to the increasedContainers + // map in NMContext. + synchronized (this.context) { + // Process container resource increase requests + for (org.apache.hadoop.yarn.api.records.Token token : + requests.getContainersToIncrease()) { + ContainerId containerId = null; + try { + if (token.getIdentifier() == null) { + throw new IOException(INVALID_CONTAINERTOKEN_MSG); + } + ContainerTokenIdentifier containerTokenIdentifier = + BuilderUtils.newContainerTokenIdentifier(token); + verifyAndGetContainerTokenIdentifier(token, + containerTokenIdentifier); + authorizeStartAndResourceIncreaseRequest( + nmTokenIdentifier, containerTokenIdentifier, false); + containerId = containerTokenIdentifier.getContainerID(); + // Reuse the startContainer logic to update NMToken, + // as container resource increase request will have come with + // an updated NMToken. + updateNMTokenIdentifier(nmTokenIdentifier); + Resource resource = containerTokenIdentifier.getResource(); + changeContainerResourceInternal(containerId, resource, true); + successfullyIncreasedContainers.add(containerId); + } catch (YarnException | InvalidToken e) { + failedContainers.put(containerId, SerializedException.newInstance(e)); + } catch (IOException e) { + throw RPCUtil.getRemoteException(e); + } + } + } + return IncreaseContainersResourceResponse.newInstance( + successfullyIncreasedContainers, failedContainers); + } + + @SuppressWarnings("unchecked") + private void changeContainerResourceInternal( + ContainerId containerId, Resource targetResource, boolean increase) + throws YarnException, IOException { + Container container = context.getContainers().get(containerId); + // Check container existence + if (container == null) { + if (nodeStatusUpdater.isContainerRecentlyStopped(containerId)) { + throw RPCUtil.getRemoteException("Container " + containerId.toString() + + " was recently stopped on node manager."); + } else { + throw RPCUtil.getRemoteException("Container " + containerId.toString() + + " is not handled by this NodeManager"); + } + } + // Check container state + org.apache.hadoop.yarn.server.nodemanager. + containermanager.container.ContainerState currentState = + container.getContainerState(); + if (currentState != org.apache.hadoop.yarn.server. + nodemanager.containermanager.container.ContainerState.RUNNING) { + throw RPCUtil.getRemoteException("Container " + containerId.toString() + + " is in " + currentState.name() + " state." + + " Resource can only be changed when a container is in" + + " RUNNING state"); + } + // Check validity of the target resource. + Resource currentResource = container.getResource(); + if (currentResource.equals(targetResource)) { + LOG.warn("Unable to change resource for container " + + containerId.toString() + + ". The target resource " + + targetResource.toString() + + " is the same as the current resource"); + return; + } + if (increase && !Resources.fitsIn(currentResource, targetResource)) { + throw RPCUtil.getRemoteException("Unable to increase resource for " + + "container " + containerId.toString() + + ". The target resource " + + targetResource.toString() + + " is smaller than the current resource " + + currentResource.toString()); + } + if (!increase && + (!Resources.fitsIn(Resources.none(), targetResource) + || !Resources.fitsIn(targetResource, currentResource))) { + throw RPCUtil.getRemoteException("Unable to decrease resource for " + + "container " + containerId.toString() + + ". The target resource " + + targetResource.toString() + + " is not smaller than the current resource " + + currentResource.toString()); + } + if (increase) { + org.apache.hadoop.yarn.api.records.Container increasedContainer = + org.apache.hadoop.yarn.api.records.Container.newInstance( + containerId, null, null, targetResource, null, null); + if (context.getIncreasedContainers().putIfAbsent(containerId, + increasedContainer) != null){ + throw RPCUtil.getRemoteException("Container " + containerId.toString() + + " resource is being increased."); + } + } + this.readLock.lock(); + try { + if (!serviceStopped) { + // Persist container resource change for recovery + this.context.getNMStateStore().storeContainerResourceChanged( + containerId, targetResource); + getContainersMonitor().handle( + new ChangeMonitoringContainerResourceEvent( + containerId, targetResource)); + } else { + throw new YarnException( + "Unable to change container resource as the NodeManager is " + + "in the process of shutting down"); + } + } finally { + this.readLock.unlock(); + } + } + @Private @VisibleForTesting protected void updateNMTokenIdentifier(NMTokenIdentifier nmTokenIdentifier) @@ -1169,6 +1334,21 @@ public void handle(ContainerManagerEvent event) { "Container Killed by ResourceManager")); } break; + case DECREASE_CONTAINERS_RESOURCE: + CMgrDecreaseContainersResourceEvent containersDecreasedEvent = + (CMgrDecreaseContainersResourceEvent) event; + for (org.apache.hadoop.yarn.api.records.Container container + : containersDecreasedEvent.getContainersToDecrease()) { + try { + changeContainerResourceInternal(container.getId(), + container.getResource(), false); + } catch (YarnException e) { + LOG.error("Unable to decrease container resource", e); + } catch (IOException e) { + LOG.error("Unable to update container resource in store", e); + } + } + break; default: throw new YarnRuntimeException( "Got an unknown ContainerManagerEvent type: " + event.getType()); 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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java index 56b4fdd..1d2ec56 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java @@ -37,6 +37,8 @@ Resource getResource(); + void setResource(Resource targetResource); + ContainerTokenIdentifier getContainerTokenIdentifier(); String getUser(); 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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java index 3c76596..eff2188 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java @@ -79,6 +79,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.hadoop.yarn.util.resource.Resources; public class ContainerImpl implements Container { @@ -91,7 +92,7 @@ private final ContainerLaunchContext launchContext; private final ContainerTokenIdentifier containerTokenIdentifier; private final ContainerId containerId; - private final Resource resource; + private volatile Resource resource; private final String user; private int exitCode = ContainerExitStatus.INVALID; private final StringBuilder diagnostics; @@ -153,13 +154,19 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher, Credentials creds, NodeManagerMetrics metrics, ContainerTokenIdentifier containerTokenIdentifier, RecoveredContainerStatus recoveredStatus, int exitCode, - String diagnostics, boolean wasKilled) { + String diagnostics, boolean wasKilled, Resource recoveredCapability) { this(conf, dispatcher, stateStore, launchContext, creds, metrics, containerTokenIdentifier); this.recoveredStatus = recoveredStatus; this.exitCode = exitCode; this.recoveredAsKilled = wasKilled; this.diagnostics.append(diagnostics); + if (recoveredCapability != null + && !this.resource.equals(recoveredCapability)) { + // resource capability had been updated before NM was down + this.resource = Resource.newInstance(recoveredCapability.getMemory(), + recoveredCapability.getVirtualCores()); + } } private static final ContainerDiagnosticsUpdateTransition UPDATE_DIAGNOSTICS_TRANSITION = @@ -249,7 +256,7 @@ public ContainerImpl(Configuration conf, Dispatcher dispatcher, ContainerEventType.KILL_CONTAINER, new KillTransition()) .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE, ContainerEventType.CONTAINER_KILLED_ON_REQUEST, - new KilledExternallyTransition()) + new KilledExternallyTransition()) // From CONTAINER_EXITED_WITH_SUCCESS State .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE, @@ -424,7 +431,7 @@ public ContainerStatus cloneAndGetContainerStatus() { this.readLock.lock(); try { return BuilderUtils.newContainerStatus(this.containerId, - getCurrentState(), diagnostics.toString(), exitCode); + getCurrentState(), diagnostics.toString(), exitCode, getResource()); } finally { this.readLock.unlock(); } @@ -451,7 +458,14 @@ public ContainerId getContainerId() { @Override public Resource getResource() { - return this.resource; + return Resources.clone(this.resource); + } + + @Override + public void setResource(Resource targetResource) { + Resource currentResource = getResource(); + this.resource = Resources.clone(targetResource); + this.metrics.changeContainer(currentResource, targetResource); } @Override 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/monitor/ChangeMonitoringContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java new file mode 100644 index 0000000..e0abbed --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java @@ -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. + */ + +package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; + +public class ChangeMonitoringContainerResourceEvent extends ContainersMonitorEvent { + private final Resource resource; + + public ChangeMonitoringContainerResourceEvent(ContainerId containerId, + Resource resource) { + super(containerId, + ContainersMonitorEventType.CHANGE_MONITORING_CONTAINER_RESOURCE); + this.resource = resource; + } + + public Resource getResource() { + return this.resource; + } +} 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/monitor/ContainersMonitorEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java index be99651..2b31480 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java @@ -20,5 +20,6 @@ public enum ContainersMonitorEventType { START_MONITORING_CONTAINER, - STOP_MONITORING_CONTAINER + STOP_MONITORING_CONTAINER, + CHANGE_MONITORING_CONTAINER_RESOURCE } 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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java index afb51ad..b3839d2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java @@ -18,13 +18,11 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,12 +30,14 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.server.api.records.ResourceUtilization; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent; import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; @@ -56,16 +56,16 @@ private boolean containerMetricsEnabled; private long containerMetricsPeriodMs; - final List containersToBeRemoved; - final Map containersToBeAdded; - Map trackingContainers = - new HashMap(); + @VisibleForTesting + final Map trackingContainers = + new ConcurrentHashMap<>(); - final ContainerExecutor containerExecutor; + private final ContainerExecutor containerExecutor; private final Dispatcher eventDispatcher; private final Context context; private ResourceCalculatorPlugin resourceCalculatorPlugin; private Configuration conf; + private static float vmemRatio; private Class processTreeClass; private long maxVmemAllottedForContainers = UNKNOWN_MEMORY_LIMIT; @@ -82,6 +82,8 @@ private ResourceUtilization containersUtilization; + private volatile boolean stopped = false; + public ContainersMonitorImpl(ContainerExecutor exec, AsyncDispatcher dispatcher, Context context) { super("containers-monitor"); @@ -90,8 +92,6 @@ public ContainersMonitorImpl(ContainerExecutor exec, this.eventDispatcher = dispatcher; this.context = context; - this.containersToBeAdded = new HashMap(); - this.containersToBeRemoved = new ArrayList(); this.monitoringThread = new MonitoringThread(); this.containersUtilization = ResourceUtilization.newInstance(0, 0, 0.0f); @@ -140,7 +140,7 @@ protected void serviceInit(Configuration conf) throws Exception { this.maxVCoresAllottedForContainers = configuredVCoresForContainers; // ///////// Virtual memory configuration ////// - float vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO, + vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO, YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO); Preconditions.checkArgument(vmemRatio > 0.99f, YarnConfiguration.NM_VMEM_PMEM_RATIO + " should be at least 1.0"); @@ -218,6 +218,7 @@ protected void serviceStart() throws Exception { @Override protected void serviceStop() throws Exception { if (containersMonitorEnabled) { + stopped = true; this.monitoringThread.interrupt(); try { this.monitoringThread.join(); @@ -228,7 +229,8 @@ protected void serviceStop() throws Exception { super.serviceStop(); } - private static class ProcessTreeInfo { + @VisibleForTesting + static class ProcessTreeInfo { private ContainerId containerId; private String pid; private ResourceCalculatorProcessTree pTree; @@ -267,26 +269,43 @@ public void setProcessTree(ResourceCalculatorProcessTree pTree) { this.pTree = pTree; } - public long getVmemLimit() { + /** + * @return Virtual memory limit for the process tree in bytes + */ + public synchronized long getVmemLimit() { return this.vmemLimit; } /** * @return Physical memory limit for the process tree in bytes */ - public long getPmemLimit() { + public synchronized long getPmemLimit() { return this.pmemLimit; } /** - * Return the number of cpu vcores assigned - * @return + * @return Number of cpu vcores assigned */ - public int getCpuVcores() { + public synchronized int getCpuVcores() { return this.cpuVcores; } - } + /** + * Set resource limit for enforcement + * @param pmemLimit + * Physical memory limit for the process tree in bytes + * @param vmemLimit + * Virtual memory limit for the process tree in bytes + * @param cpuVcores + * Number of cpu vcores assigned + */ + public synchronized void setResourceLimit( + long pmemLimit, long vmemLimit, int cpuVcores) { + this.pmemLimit = pmemLimit; + this.vmemLimit = vmemLimit; + this.cpuVcores = cpuVcores; + } + } /** * Check whether a container's process tree's current memory usage is over @@ -359,8 +378,7 @@ public MonitoringThread() { @Override public void run() { - while (true) { - + while (!stopped && !Thread.currentThread().isInterrupted()) { // Print the processTrees for debugging. if (LOG.isDebugEnabled()) { StringBuilder tmp = new StringBuilder("[ "); @@ -372,31 +390,6 @@ public void run() { + tmp.substring(0, tmp.length()) + "]"); } - // Add new containers - synchronized (containersToBeAdded) { - for (Entry entry : containersToBeAdded - .entrySet()) { - ContainerId containerId = entry.getKey(); - ProcessTreeInfo processTreeInfo = entry.getValue(); - LOG.info("Starting resource-monitoring for " + containerId); - trackingContainers.put(containerId, processTreeInfo); - } - containersToBeAdded.clear(); - } - - // Remove finished containers - synchronized (containersToBeRemoved) { - for (ContainerId containerId : containersToBeRemoved) { - if (containerMetricsEnabled) { - ContainerMetrics.forContainer( - containerId, containerMetricsPeriodMs).finished(); - } - trackingContainers.remove(containerId); - LOG.info("Stopping resource-monitoring for " + containerId); - } - containersToBeRemoved.clear(); - } - // Temporary structure to calculate the total resource utilization of // the containers ResourceUtilization trackedContainersUtilization = @@ -408,10 +401,8 @@ public void run() { long pmemByAllContainers = 0; long cpuUsagePercentPerCoreByAllContainers = 0; long cpuUsageTotalCoresByAllContainers = 0; - for (Iterator> it = - trackingContainers.entrySet().iterator(); it.hasNext();) { - - Map.Entry entry = it.next(); + for (Entry entry : trackingContainers + .entrySet()) { ContainerId containerId = entry.getKey(); ProcessTreeInfo ptInfo = entry.getValue(); try { @@ -435,11 +426,6 @@ public void run() { if (containerMetricsEnabled) { ContainerMetrics usageMetrics = ContainerMetrics .forContainer(containerId, containerMetricsPeriodMs); - int cpuVcores = ptInfo.getCpuVcores(); - final int vmemLimit = (int) (ptInfo.getVmemLimit() >> 20); - final int pmemLimit = (int) (ptInfo.getPmemLimit() >> 20); - usageMetrics.recordResourceLimit( - vmemLimit, pmemLimit, cpuVcores); usageMetrics.recordProcessId(pId); } } @@ -548,7 +534,7 @@ public void run() { eventDispatcher.getEventHandler().handle( new ContainerKillEvent(containerId, containerExitStatus, msg)); - it.remove(); + trackingContainers.remove(containerId); LOG.info("Removed ProcessTree with root " + pId); } } catch (Exception e) { @@ -605,6 +591,60 @@ private String formatUsageString(long currentVmemUsage, long vmemLimit, } } + private void changeContainerResource( + ContainerId containerId, Resource resource) { + Container container = context.getContainers().get(containerId); + // Check container existence + if (container == null) { + LOG.warn("Container " + containerId.toString() + "does not exist"); + return; + } + container.setResource(resource); + } + + private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) { + if (!containerMetricsEnabled || monitoringEvent == null) { + return; + } + + ContainerId containerId = monitoringEvent.getContainerId(); + ContainerMetrics usageMetrics = ContainerMetrics + .forContainer(containerId, containerMetricsPeriodMs); + + int vmemLimitMBs; + int pmemLimitMBs; + int cpuVcores; + switch (monitoringEvent.getType()) { + case START_MONITORING_CONTAINER: + ContainerStartMonitoringEvent startEvent = + (ContainerStartMonitoringEvent) monitoringEvent; + usageMetrics.recordStateChangeDurations( + startEvent.getLaunchDuration(), + startEvent.getLocalizationDuration()); + cpuVcores = startEvent.getCpuVcores(); + vmemLimitMBs = (int) (startEvent.getVmemLimit() >> 20); + pmemLimitMBs = (int) (startEvent.getPmemLimit() >> 20); + usageMetrics.recordResourceLimit( + vmemLimitMBs, pmemLimitMBs, cpuVcores); + break; + case STOP_MONITORING_CONTAINER: + usageMetrics.finished(); + break; + case CHANGE_MONITORING_CONTAINER_RESOURCE: + ChangeMonitoringContainerResourceEvent changeEvent = + (ChangeMonitoringContainerResourceEvent) monitoringEvent; + Resource resource = changeEvent.getResource(); + pmemLimitMBs = resource.getMemory(); + vmemLimitMBs = (int) (pmemLimitMBs * vmemRatio); + cpuVcores = resource.getVirtualCores(); + usageMetrics.recordResourceLimit( + vmemLimitMBs, pmemLimitMBs, cpuVcores); + break; + default: + break; + } + } + @Override public long getVmemAllocatedForContainers() { return this.maxVmemAllottedForContainers; @@ -650,38 +690,53 @@ public void setContainersUtilization(ResourceUtilization utilization) { } @Override + @SuppressWarnings("unchecked") public void handle(ContainersMonitorEvent monitoringEvent) { - + ContainerId containerId = monitoringEvent.getContainerId(); if (!containersMonitorEnabled) { + if (monitoringEvent.getType() == ContainersMonitorEventType + .CHANGE_MONITORING_CONTAINER_RESOURCE) { + // Nothing to enforce. Update container resource immediately. + ChangeMonitoringContainerResourceEvent changeEvent = + (ChangeMonitoringContainerResourceEvent) monitoringEvent; + changeContainerResource(containerId, changeEvent.getResource()); + } return; } - ContainerId containerId = monitoringEvent.getContainerId(); switch (monitoringEvent.getType()) { case START_MONITORING_CONTAINER: ContainerStartMonitoringEvent startEvent = (ContainerStartMonitoringEvent) monitoringEvent; - - if (containerMetricsEnabled) { - ContainerMetrics usageMetrics = ContainerMetrics - .forContainer(containerId, containerMetricsPeriodMs); - usageMetrics.recordStateChangeDurations( - startEvent.getLaunchDuration(), - startEvent.getLocalizationDuration()); - } - - synchronized (this.containersToBeAdded) { - ProcessTreeInfo processTreeInfo = - new ProcessTreeInfo(containerId, null, null, - startEvent.getVmemLimit(), startEvent.getPmemLimit(), - startEvent.getCpuVcores()); - this.containersToBeAdded.put(containerId, processTreeInfo); - } + LOG.info("Starting resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + trackingContainers.put(containerId, + new ProcessTreeInfo(containerId, null, null, + startEvent.getVmemLimit(), startEvent.getPmemLimit(), + startEvent.getCpuVcores())); break; case STOP_MONITORING_CONTAINER: - synchronized (this.containersToBeRemoved) { - this.containersToBeRemoved.add(containerId); + LOG.info("Stopping resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + trackingContainers.remove(containerId); + break; + case CHANGE_MONITORING_CONTAINER_RESOURCE: + ChangeMonitoringContainerResourceEvent changeEvent = + (ChangeMonitoringContainerResourceEvent) monitoringEvent; + ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId); + if (processTreeInfo == null) { + LOG.warn("Failed to track container " + + containerId.toString() + + ". It may have already completed."); + break; } + LOG.info("Changing resource-monitoring for " + containerId); + updateContainerMetrics(monitoringEvent); + long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L; + long vmemLimit = (long) (pmemLimit * vmemRatio); + int cpuVcores = changeEvent.getResource().getVirtualCores(); + processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores); + changeContainerResource(containerId, changeEvent.getResource()); break; default: // TODO: Wrong event. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java index 56797d1..a38d0b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java @@ -133,6 +133,17 @@ public void releaseContainer(Resource res) { availableVCores.incr(res.getVirtualCores()); } + public void changeContainer(Resource before, Resource now) { + int deltaMB = now.getMemory() - before.getMemory(); + int deltaVCores = now.getVirtualCores() - before.getVirtualCores(); + allocatedMB = allocatedMB + deltaMB; + allocatedGB.set((int)Math.ceil(allocatedMB/1024d)); + availableMB = availableMB - deltaMB; + availableGB.set((int)Math.floor(availableMB/1024d)); + allocatedVCores.incr(deltaVCores); + availableVCores.decr(deltaVCores); + } + public void addResource(Resource res) { availableMB = availableMB + res.getMemory(); availableGB.incr((int)Math.floor(availableMB/1024d)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java index df58182..89c71bb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java @@ -40,7 +40,10 @@ 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.Resource; +import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto; @@ -99,6 +102,8 @@ private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request"; private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics"; private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched"; + private static final String CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX = + "/resourceChanged"; private static final String CONTAINER_KILLED_KEY_SUFFIX = "/killed"; private static final String CONTAINER_EXIT_CODE_KEY_SUFFIX = "/exitcode"; @@ -230,6 +235,9 @@ private RecoveredContainerState loadContainerState(ContainerId containerId, } else if (suffix.equals(CONTAINER_EXIT_CODE_KEY_SUFFIX)) { rcs.status = RecoveredContainerStatus.COMPLETED; rcs.exitCode = Integer.parseInt(asString(entry.getValue())); + } else if (suffix.equals(CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX)) { + rcs.capability = new ResourcePBImpl( + ResourceProto.parseFrom(entry.getValue())); } else { throw new IOException("Unexpected container state key: " + key); } @@ -275,6 +283,20 @@ public void storeContainerLaunched(ContainerId containerId) } @Override + public void storeContainerResourceChanged(ContainerId containerId, + Resource capability) throws IOException { + String key = CONTAINERS_KEY_PREFIX + containerId.toString() + + CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX; + try { + // New value will overwrite old values for the same key + db.put(bytes(key), + ((ResourcePBImpl) capability).getProto().toByteArray()); + } catch (DBException e) { + throw new IOException(e); + } + } + + @Override public void storeContainerKilled(ContainerId containerId) throws IOException { String key = CONTAINERS_KEY_PREFIX + containerId.toString() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java index ab49543..d5dce9b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java @@ -27,6 +27,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.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto; @@ -88,6 +89,11 @@ public void storeContainerLaunched(ContainerId containerId) } @Override + public void storeContainerResourceChanged(ContainerId containerId, + Resource capability) throws IOException { + } + + @Override public void storeContainerKilled(ContainerId containerId) throws IOException { } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java index fa66349..e8ccf54 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java @@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto; @@ -74,6 +75,7 @@ public NMStateStoreService(String name) { boolean killed = false; String diagnostics = ""; StartContainerRequest startRequest; + Resource capability; public RecoveredContainerStatus getStatus() { return status; @@ -94,6 +96,10 @@ public String getDiagnostics() { public StartContainerRequest getStartRequest() { return startRequest; } + + public Resource getCapability() { + return capability; + } } public static class LocalResourceTrackerState { @@ -284,6 +290,15 @@ public abstract void storeContainerLaunched(ContainerId containerId) throws IOException; /** + * Record that a container resource has been changed + * @param containerId the container ID + * @param capability the container resource capability + * @throws IOException + */ + public abstract void storeContainerResourceChanged(ContainerId containerId, + Resource capability) throws IOException; + + /** * Record that a container has completed * @param containerId the container ID * @param exitCode the exit code from the container diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java index 349340b..3ff04d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java @@ -191,8 +191,10 @@ public void setBlockNewContainerRequests(boolean blockNewContainerRequests) { } @Override - protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier, - ContainerTokenIdentifier containerTokenIdentifier) throws YarnException { + protected void authorizeStartAndResourceIncreaseRequest( + NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + boolean startRequest) throws YarnException { // do nothing } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java index a47e7f7..75bcdae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java @@ -189,6 +189,39 @@ public void testStartContainerFailureWithUnknownAuxService() throws Exception { super.testStartContainerFailureWithUnknownAuxService(); } + @Override + public void testIncreaseContainerResourceWithInvalidRequests() throws Exception { + // Don't run the test if the binary is not available. + if (!shouldRunTest()) { + LOG.info("LCE binary path is not passed. Not running the test"); + return; + } + LOG.info("Running testIncreaseContainerResourceWithInvalidRequests"); + super.testIncreaseContainerResourceWithInvalidRequests(); + } + + @Override + public void testIncreaseContainerResourceWithInvalidResource() throws Exception { + // Don't run the test if the binary is not available. + if (!shouldRunTest()) { + LOG.info("LCE binary path is not passed. Not running the test"); + return; + } + LOG.info("Running testIncreaseContainerResourceWithInvalidResource"); + super.testIncreaseContainerResourceWithInvalidResource(); + } + + @Override + public void testChangeContainerResource() throws Exception { + // Don't run the test if the binary is not available. + if (!shouldRunTest()) { + LOG.info("LCE binary path is not passed. Not running the test"); + return; + } + LOG.info("Running testChangeContainerResource"); + super.testChangeContainerResource(); + } + private boolean shouldRunTest() { return System .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java index c22d475..4250ac3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java @@ -18,21 +18,35 @@ package org.apache.hadoop.yarn.server.nodemanager; +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; +import java.io.PrintWriter; +import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.UnsupportedFileSystemException; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.SecretManager; +import org.apache.hadoop.util.Shell; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -41,8 +55,13 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException; @@ -50,6 +69,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; +import org.apache.hadoop.yarn.security.NMTokenIdentifier; import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest; @@ -57,12 +78,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.records.NodeAction; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager; 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.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils; +import org.apache.hadoop.yarn.util.ConverterUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -87,7 +111,10 @@ private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false); private final NodeManagerEvent resyncEvent = new NodeManagerEvent(NodeManagerEventType.RESYNC); + private final long DUMMY_RM_IDENTIFIER = 1234; + protected static Log LOG = LogFactory + .getLog(TestNodeManagerResync.class); @Before public void setup() throws UnsupportedFileSystemException { @@ -209,6 +236,32 @@ public void testNMshutdownWhenResyncThrowException() throws IOException, nm.stop(); } + @SuppressWarnings("unchecked") + @Test(timeout=60000) + public void testContainerResourceIncreaseIsSynchronizedWithRMResync() + throws IOException, InterruptedException, YarnException { + NodeManager nm = new TestNodeManager4(); + YarnConfiguration conf = createNMConfig(); + conf.setBoolean( + YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true); + nm.init(conf); + nm.start(); + // Start a container and make sure it is in RUNNING state + ((TestNodeManager4)nm).startContainer(); + // Simulate a container resource increase in a separate thread + ((TestNodeManager4)nm).increaseContainersResource(); + // Simulate RM restart by sending a RESYNC event + LOG.info("Sending out RESYNC event"); + nm.getNMDispatcher().getEventHandler().handle( + new NodeManagerEvent(NodeManagerEventType.RESYNC)); + try { + syncBarrier.await(); + } catch (BrokenBarrierException e) { + e.printStackTrace(); + } + Assert.assertFalse(assertionFailedInThread.get()); + nm.stop(); + } // This is to test when NM gets the resync response from last heart beat, it // should be able to send the already-sent-via-last-heart-beat container @@ -588,6 +641,211 @@ protected void registerWithRM() throws YarnException, IOException { } }} + class TestNodeManager4 extends NodeManager { + + private Thread increaseContainerResourceThread = null; + + @Override + protected NodeStatusUpdater createNodeStatusUpdater(Context context, + Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { + return new TestNodeStatusUpdaterImpl4(context, dispatcher, + healthChecker, metrics); + } + + @Override + protected ContainerManagerImpl createContainerManager(Context context, + ContainerExecutor exec, DeletionService del, + NodeStatusUpdater nodeStatusUpdater, + ApplicationACLsManager aclsManager, + LocalDirsHandlerService dirsHandler) { + return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater, + metrics, dirsHandler){ + @Override + public void + setBlockNewContainerRequests(boolean blockNewContainerRequests) { + // do nothing + } + + @Override + protected void authorizeGetAndStopContainerRequest( + ContainerId containerId, Container container, + boolean stopRequest, NMTokenIdentifier identifier) + throws YarnException { + // do nothing + } + @Override + protected void authorizeUser(UserGroupInformation remoteUgi, + NMTokenIdentifier nmTokenIdentifier) { + // do nothing + } + @Override + protected void authorizeStartAndResourceIncreaseRequest( + NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + boolean startRequest) throws YarnException { + try { + // Sleep 2 seconds to simulate a pro-longed increase action. + // If during this time a RESYNC event is sent by RM, the + // resync action should block until the increase action is + // completed. + // See testContainerResourceIncreaseIsSynchronizedWithRMResync() + Thread.sleep(2000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + @Override + protected void updateNMTokenIdentifier( + NMTokenIdentifier nmTokenIdentifier) + throws SecretManager.InvalidToken { + // Do nothing + } + @Override + public Map getAuxServiceMetaData() { + return new HashMap<>(); + } + @Override + protected NMTokenIdentifier selectNMTokenIdentifier( + UserGroupInformation remoteUgi) { + return new NMTokenIdentifier(); + } + }; + } + + // Start a container in NM + public void startContainer() + throws IOException, InterruptedException, YarnException { + LOG.info("Start a container and wait until it is in RUNNING state"); + File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile"); + PrintWriter fileWriter = new PrintWriter(scriptFile); + if (Shell.WINDOWS) { + fileWriter.println("@ping -n 100 127.0.0.1 >nul"); + } else { + fileWriter.write("\numask 0"); + fileWriter.write("\nexec sleep 100"); + } + fileWriter.close(); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + URL resource_alpha = + ConverterUtils.getYarnUrlFromPath(localFS + .makeQualified(new Path(scriptFile.getAbsolutePath()))); + LocalResource rsrc_alpha = + recordFactory.newRecordInstance(LocalResource.class); + rsrc_alpha.setResource(resource_alpha); + rsrc_alpha.setSize(-1); + rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); + rsrc_alpha.setType(LocalResourceType.FILE); + rsrc_alpha.setTimestamp(scriptFile.lastModified()); + String destinationFile = "dest_file"; + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); + List commands = + Arrays.asList(Shell.getRunScriptCommand(scriptFile)); + containerLaunchContext.setCommands(commands); + Resource resource = Resource.newInstance(1024, 1); + StartContainerRequest scRequest = + StartContainerRequest.newInstance( + containerLaunchContext, + getContainerToken(resource)); + List list = new ArrayList(); + list.add(scRequest); + StartContainersRequest allRequests = + StartContainersRequest.newInstance(list); + getContainerManager().startContainers(allRequests); + // Make sure the container reaches RUNNING state + ContainerId cId = TestContainerManager.createContainerId(0); + BaseContainerManagerTest.waitForNMContainerState( + getContainerManager(), cId, + org.apache.hadoop.yarn.server.nodemanager. + containermanager.container.ContainerState.RUNNING); + } + + // Increase container resource in a thread + public void increaseContainersResource() + throws InterruptedException { + LOG.info("Increase a container resource in a separate thread"); + increaseContainerResourceThread = new IncreaseContainersResourceThread(); + increaseContainerResourceThread.start(); + } + + class TestNodeStatusUpdaterImpl4 extends MockNodeStatusUpdater { + + public TestNodeStatusUpdaterImpl4(Context context, Dispatcher dispatcher, + NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) { + super(context, dispatcher, healthChecker, metrics); + } + + @Override + protected void rebootNodeStatusUpdaterAndRegisterWithRM() { + try { + try { + // Check status before registerWithRM + List containerIds = new ArrayList<>(); + ContainerId cId = TestContainerManager.createContainerId(0); + containerIds.add(cId); + GetContainerStatusesRequest gcsRequest = + GetContainerStatusesRequest.newInstance(containerIds); + ContainerStatus containerStatus = getContainerManager() + .getContainerStatuses(gcsRequest).getContainerStatuses().get(0); + assertEquals(Resource.newInstance(1024, 1), + containerStatus.getCapability()); + // Call the actual rebootNodeStatusUpdaterAndRegisterWithRM(). + // This function should be synchronized with + // increaseContainersResource(). + super.rebootNodeStatusUpdaterAndRegisterWithRM(); + // Check status after registerWithRM + containerStatus = getContainerManager() + .getContainerStatuses(gcsRequest).getContainerStatuses().get(0); + assertEquals(Resource.newInstance(4096, 2), + containerStatus.getCapability()); + } catch (AssertionError ae) { + ae.printStackTrace(); + assertionFailedInThread.set(true); + } finally { + syncBarrier.await(); + } + } catch (Exception e) { + e.printStackTrace(); + } + } + } + + class IncreaseContainersResourceThread extends Thread { + @Override + public void run() { + // Construct container resource increase request + List increaseTokens = new ArrayList(); + // Add increase request. + Resource targetResource = Resource.newInstance(4096, 2); + try { + increaseTokens.add(getContainerToken(targetResource)); + IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest.newInstance(increaseTokens); + IncreaseContainersResourceResponse increaseResponse = + getContainerManager() + .increaseContainersResource(increaseRequest); + Assert.assertEquals( + 1, increaseResponse.getSuccessfullyIncreasedContainers() + .size()); + Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty()); + } catch (Exception e) { + e.printStackTrace(); + } + } + } + + private Token getContainerToken(Resource resource) throws IOException { + ContainerId cId = TestContainerManager.createContainerId(0); + return TestContainerManager.createContainerToken( + cId, DUMMY_RM_IDENTIFIER, + getNMContext().getNodeId(), user, resource, + getNMContext().getContainerTokenSecretManager(), null); + } + } + public static NMContainerStatus createNMContainerStatus(int id, ContainerState containerState) { ApplicationId applicationId = ApplicationId.newInstance(0, 1); 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 3c0368b..70a8f55 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 @@ -1662,7 +1662,7 @@ public static ContainerStatus createContainerStatus(int id, ContainerStatus containerStatus = BuilderUtils.newContainerStatus(contaierId, containerState, "test_containerStatus: id=" + id + ", containerState: " - + containerState, 0); + + containerState, 0, Resource.newInstance(1024, 1)); return containerStatus; } 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 964379a..9bc23f6 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 @@ -620,6 +620,11 @@ public int getHttpPort() { } @Override + public ConcurrentMap getIncreasedContainers() { + return null; + } + + @Override public NMContainerTokenSecretManager getContainerTokenSecretManager() { return null; } 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/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java index 7573a7a..f482784 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java @@ -93,8 +93,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease; -import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.NMToken; @@ -292,8 +290,8 @@ public AllocateResponse allocate(AllocateRequest request) new ArrayList(), containerList, new ArrayList(), null, AMCommand.AM_RESYNC, 1, null, new ArrayList(), - new ArrayList(), - new ArrayList()); + new ArrayList(), + new ArrayList()); } @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/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 2810662..3938342 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 @@ -209,12 +209,13 @@ protected void authorizeUser(UserGroupInformation remoteUgi, // do nothing } @Override - protected void authorizeStartRequest( - NMTokenIdentifier nmTokenIdentifier, - ContainerTokenIdentifier containerTokenIdentifier) throws YarnException { - // do nothing - } - + protected void authorizeStartAndResourceIncreaseRequest( + NMTokenIdentifier nmTokenIdentifier, + ContainerTokenIdentifier containerTokenIdentifier, + boolean startRequest) throws YarnException { + // do nothing + } + @Override protected void updateNMTokenIdentifier( NMTokenIdentifier nmTokenIdentifier) throws InvalidToken { @@ -310,4 +311,34 @@ static void waitForApplicationState(ContainerManagerImpl containerManager, app.getApplicationState().equals(finalState)); } + public static void waitForNMContainerState(ContainerManagerImpl + containerManager, ContainerId containerID, + org.apache.hadoop.yarn.server.nodemanager.containermanager + .container.ContainerState finalState) + throws InterruptedException, YarnException, IOException { + waitForNMContainerState(containerManager, containerID, finalState, 20); + } + + public static void waitForNMContainerState(ContainerManagerImpl + containerManager, ContainerId containerID, + org.apache.hadoop.yarn.server.nodemanager.containermanager + .container.ContainerState finalState, int timeOutMax) + throws InterruptedException, YarnException, IOException { + Container container = + containerManager.getContext().getContainers().get(containerID); + org.apache.hadoop.yarn.server.nodemanager + .containermanager.container.ContainerState currentState = + container.getContainerState(); + int timeoutSecs = 0; + while (!currentState.equals(finalState) + && timeoutSecs++ < timeOutMax) { + Thread.sleep(1000); + LOG.info("Waiting for NM container to get into state " + finalState + + ". Current state is " + currentState); + currentState = container.getContainerState(); + } + LOG.info("Container state is " + currentState); + Assert.assertEquals("ContainerState is not correct (timedout)", + finalState, currentState); + } } 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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java index e508424..3fb4112 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java @@ -38,6 +38,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.Service; import org.apache.hadoop.util.Shell; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -72,6 +74,7 @@ import org.apache.hadoop.yarn.security.NMTokenIdentifier; import org.apache.hadoop.yarn.server.api.ResourceManagerConstants; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; +import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA; @@ -87,6 +90,8 @@ import org.junit.Test; import org.mockito.Mockito; +import static org.junit.Assert.assertEquals; + public class TestContainerManager extends BaseContainerManagerTest { public TestContainerManager() throws UnsupportedFileSystemException { @@ -103,7 +108,7 @@ public void setup() throws IOException { super.setup(); } - private ContainerId createContainerId(int id) { + public static ContainerId createContainerId(int id) { ApplicationId appId = ApplicationId.newInstance(0, 0); ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1); @@ -803,7 +808,8 @@ public void testNullTokens() throws Exception { metrics, dirsHandler); String strExceptionMsg = ""; try { - cMgrImpl.authorizeStartRequest(null, new ContainerTokenIdentifier()); + cMgrImpl.authorizeStartAndResourceIncreaseRequest( + null, new ContainerTokenIdentifier(), true); } catch(YarnException ye) { strExceptionMsg = ye.getMessage(); } @@ -812,7 +818,8 @@ public void testNullTokens() throws Exception { strExceptionMsg = ""; try { - cMgrImpl.authorizeStartRequest(new NMTokenIdentifier(), null); + cMgrImpl.authorizeStartAndResourceIncreaseRequest( + new NMTokenIdentifier(), null, true); } catch(YarnException ye) { strExceptionMsg = ye.getMessage(); } @@ -878,6 +885,263 @@ public void testNullTokens() throws Exception { ContainerManagerImpl.INVALID_CONTAINERTOKEN_MSG); } + @Test + public void testIncreaseContainerResourceWithInvalidRequests() throws Exception { + containerManager.start(); + // Start 4 containers 0..4 with default resource (1024, 1) + List list = new ArrayList<>(); + ContainerLaunchContext containerLaunchContext = recordFactory + .newRecordInstance(ContainerLaunchContext.class); + for (int i = 0; i < 4; i++) { + ContainerId cId = createContainerId(i); + long identifier = DUMMY_RM_IDENTIFIER; + Token containerToken = createContainerToken(cId, identifier, + context.getNodeId(), user, context.getContainerTokenSecretManager()); + StartContainerRequest request = StartContainerRequest.newInstance( + containerLaunchContext, containerToken); + list.add(request); + } + StartContainersRequest requestList = StartContainersRequest + .newInstance(list); + StartContainersResponse response = containerManager + .startContainers(requestList); + + Assert.assertEquals(4, response.getSuccessfullyStartedContainers().size()); + int i = 0; + for (ContainerId id : response.getSuccessfullyStartedContainers()) { + Assert.assertEquals(i, id.getContainerId()); + i++; + } + + Thread.sleep(2000); + // Construct container resource increase request, + List increaseTokens = new ArrayList(); + // Add increase request for container-0, the request will fail as the + // container will have exited, and won't be in RUNNING state + ContainerId cId0 = createContainerId(0); + Token containerToken = + createContainerToken(cId0, DUMMY_RM_IDENTIFIER, + context.getNodeId(), user, + Resource.newInstance(1234, 3), + context.getContainerTokenSecretManager(), null); + increaseTokens.add(containerToken); + // Add increase request for container-7, the request will fail as the + // container does not exist + ContainerId cId7 = createContainerId(7); + containerToken = + createContainerToken(cId7, DUMMY_RM_IDENTIFIER, + context.getNodeId(), user, + Resource.newInstance(1234, 3), + context.getContainerTokenSecretManager(), null); + increaseTokens.add(containerToken); + + IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest + .newInstance(increaseTokens); + IncreaseContainersResourceResponse increaseResponse = + containerManager.increaseContainersResource(increaseRequest); + // Check response + Assert.assertEquals( + 0, increaseResponse.getSuccessfullyIncreasedContainers().size()); + Assert.assertEquals(2, increaseResponse.getFailedRequests().size()); + for (Map.Entry entry : increaseResponse + .getFailedRequests().entrySet()) { + Assert.assertNotNull("Failed message", entry.getValue().getMessage()); + if (cId0.equals(entry.getKey())) { + Assert.assertTrue(entry.getValue().getMessage() + .contains("Resource can only be changed when a " + + "container is in RUNNING state")); + } else if (cId7.equals(entry.getKey())) { + Assert.assertTrue(entry.getValue().getMessage() + .contains("Container " + cId7.toString() + + " is not handled by this NodeManager")); + } else { + throw new YarnException("Received failed request from wrong" + + " container: " + entry.getKey().toString()); + } + } + } + + @Test + public void testIncreaseContainerResourceWithInvalidResource() throws Exception { + containerManager.start(); + File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile"); + PrintWriter fileWriter = new PrintWriter(scriptFile); + // Construct the Container-id + ContainerId cId = createContainerId(0); + if (Shell.WINDOWS) { + fileWriter.println("@ping -n 100 127.0.0.1 >nul"); + } else { + fileWriter.write("\numask 0"); + fileWriter.write("\nexec sleep 100"); + } + fileWriter.close(); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + URL resource_alpha = + ConverterUtils.getYarnUrlFromPath(localFS + .makeQualified(new Path(scriptFile.getAbsolutePath()))); + LocalResource rsrc_alpha = + recordFactory.newRecordInstance(LocalResource.class); + rsrc_alpha.setResource(resource_alpha); + rsrc_alpha.setSize(-1); + rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); + rsrc_alpha.setType(LocalResourceType.FILE); + rsrc_alpha.setTimestamp(scriptFile.lastModified()); + String destinationFile = "dest_file"; + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); + List commands = + Arrays.asList(Shell.getRunScriptCommand(scriptFile)); + containerLaunchContext.setCommands(commands); + + StartContainerRequest scRequest = + StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), + user, context.getContainerTokenSecretManager())); + List list = new ArrayList(); + list.add(scRequest); + StartContainersRequest allRequests = + StartContainersRequest.newInstance(list); + containerManager.startContainers(allRequests); + // Make sure the container reaches RUNNING state + BaseContainerManagerTest.waitForNMContainerState(containerManager, cId, + org.apache.hadoop.yarn.server.nodemanager. + containermanager.container.ContainerState.RUNNING); + // Construct container resource increase request, + List increaseTokens = new ArrayList(); + // Add increase request. The increase request should fail + // as the current resource does not fit in the target resource + Token containerToken = + createContainerToken(cId, DUMMY_RM_IDENTIFIER, + context.getNodeId(), user, + Resource.newInstance(512, 1), + context.getContainerTokenSecretManager(), null); + increaseTokens.add(containerToken); + IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest + .newInstance(increaseTokens); + IncreaseContainersResourceResponse increaseResponse = + containerManager.increaseContainersResource(increaseRequest); + // Check response + Assert.assertEquals( + 0, increaseResponse.getSuccessfullyIncreasedContainers().size()); + Assert.assertEquals(1, increaseResponse.getFailedRequests().size()); + for (Map.Entry entry : increaseResponse + .getFailedRequests().entrySet()) { + if (cId.equals(entry.getKey())) { + Assert.assertNotNull("Failed message", entry.getValue().getMessage()); + Assert.assertTrue(entry.getValue().getMessage() + .contains("The target resource " + + Resource.newInstance(512, 1).toString() + + " is smaller than the current resource " + + Resource.newInstance(1024, 1))); + } else { + throw new YarnException("Received failed request from wrong" + + " container: " + entry.getKey().toString()); + } + } + } + + @Test + public void testChangeContainerResource() throws Exception { + containerManager.start(); + File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile"); + PrintWriter fileWriter = new PrintWriter(scriptFile); + // Construct the Container-id + ContainerId cId = createContainerId(0); + if (Shell.WINDOWS) { + fileWriter.println("@ping -n 100 127.0.0.1 >nul"); + } else { + fileWriter.write("\numask 0"); + fileWriter.write("\nexec sleep 100"); + } + fileWriter.close(); + ContainerLaunchContext containerLaunchContext = + recordFactory.newRecordInstance(ContainerLaunchContext.class); + URL resource_alpha = + ConverterUtils.getYarnUrlFromPath(localFS + .makeQualified(new Path(scriptFile.getAbsolutePath()))); + LocalResource rsrc_alpha = + recordFactory.newRecordInstance(LocalResource.class); + rsrc_alpha.setResource(resource_alpha); + rsrc_alpha.setSize(-1); + rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); + rsrc_alpha.setType(LocalResourceType.FILE); + rsrc_alpha.setTimestamp(scriptFile.lastModified()); + String destinationFile = "dest_file"; + Map localResources = + new HashMap(); + localResources.put(destinationFile, rsrc_alpha); + containerLaunchContext.setLocalResources(localResources); + List commands = + Arrays.asList(Shell.getRunScriptCommand(scriptFile)); + containerLaunchContext.setCommands(commands); + StartContainerRequest scRequest = + StartContainerRequest.newInstance( + containerLaunchContext, + createContainerToken(cId, DUMMY_RM_IDENTIFIER, + context.getNodeId(), user, + context.getContainerTokenSecretManager())); + List list = new ArrayList(); + list.add(scRequest); + StartContainersRequest allRequests = + StartContainersRequest.newInstance(list); + containerManager.startContainers(allRequests); + // Make sure the container reaches RUNNING state + BaseContainerManagerTest.waitForNMContainerState(containerManager, cId, + org.apache.hadoop.yarn.server.nodemanager. + containermanager.container.ContainerState.RUNNING); + // Construct container resource increase request, + List increaseTokens = new ArrayList(); + // Add increase request. + Resource targetResource = Resource.newInstance(4096, 2); + Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER, + context.getNodeId(), user, targetResource, + context.getContainerTokenSecretManager(), null); + increaseTokens.add(containerToken); + IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest.newInstance(increaseTokens); + IncreaseContainersResourceResponse increaseResponse = + containerManager.increaseContainersResource(increaseRequest); + Assert.assertEquals( + 1, increaseResponse.getSuccessfullyIncreasedContainers().size()); + Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty()); + // Check status + List containerIds = new ArrayList<>(); + containerIds.add(cId); + GetContainerStatusesRequest gcsRequest = + GetContainerStatusesRequest.newInstance(containerIds); + ContainerStatus containerStatus = containerManager + .getContainerStatuses(gcsRequest).getContainerStatuses().get(0); + // Check status immediately as resource increase is blocking + assertEquals(targetResource, containerStatus.getCapability()); + // Simulate a decrease request + List containersToDecrease + = new ArrayList<>(); + targetResource = Resource.newInstance(2048, 2); + org.apache.hadoop.yarn.api.records.Container decreasedContainer = + org.apache.hadoop.yarn.api.records.Container + .newInstance(cId, null, null, targetResource, null, null); + containersToDecrease.add(decreasedContainer); + containerManager.handle( + new CMgrDecreaseContainersResourceEvent(containersToDecrease)); + // Check status with retry + containerStatus = containerManager + .getContainerStatuses(gcsRequest).getContainerStatuses().get(0); + int retry = 0; + while (!targetResource.equals(containerStatus.getCapability()) && + (retry++ < 5)) { + Thread.sleep(200); + containerStatus = containerManager.getContainerStatuses(gcsRequest) + .getContainerStatuses().get(0); + } + assertEquals(targetResource, containerStatus.getCapability()); + } + public static Token createContainerToken(ContainerId cId, long rmIdentifier, NodeId nodeId, String user, NMContainerTokenSecretManager containerTokenSecretManager) @@ -892,15 +1156,21 @@ public static Token createContainerToken(ContainerId cId, long rmIdentifier, LogAggregationContext logAggregationContext) throws IOException { Resource r = BuilderUtils.newResource(1024, 1); + return createContainerToken(cId, rmIdentifier, nodeId, user, r, + containerTokenSecretManager, logAggregationContext); + } + + public static Token createContainerToken(ContainerId cId, long rmIdentifier, + NodeId nodeId, String user, Resource resource, + NMContainerTokenSecretManager containerTokenSecretManager, + LogAggregationContext logAggregationContext) + throws IOException { ContainerTokenIdentifier containerTokenIdentifier = - new ContainerTokenIdentifier(cId, nodeId.toString(), user, r, + new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier, Priority.newInstance(0), 0, logAggregationContext, null); - Token containerToken = - BuilderUtils - .newContainerToken(nodeId, containerTokenSecretManager - .retrievePassword(containerTokenIdentifier), + return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager + .retrievePassword(containerTokenIdentifier), containerTokenIdentifier); - return 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/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 4d0aacd..43f1b29 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 @@ -28,18 +28,30 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; import java.nio.ByteBuffer; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Shell; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse; @@ -48,9 +60,17 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.LogAggregationContext; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.NMTokenIdentifier; import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl; @@ -58,6 +78,9 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.DeletionService; +import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; +import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService; +import org.apache.hadoop.yarn.server.nodemanager.NodeManager; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; @@ -65,6 +88,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher; import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; @@ -77,18 +101,50 @@ 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; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.junit.Before; import org.junit.Test; -public class TestContainerManagerRecovery { +public class TestContainerManagerRecovery extends BaseContainerManagerTest { - private NodeManagerMetrics metrics = NodeManagerMetrics.create(); + public TestContainerManagerRecovery() throws UnsupportedFileSystemException { + super(); + } + + @Override + @Before + public void setup() throws IOException { + localFS.delete(new Path(localDir.getAbsolutePath()), true); + localFS.delete(new Path(tmpDir.getAbsolutePath()), true); + localFS.delete(new Path(localLogDir.getAbsolutePath()), true); + localFS.delete(new Path(remoteLogDir.getAbsolutePath()), true); + localDir.mkdir(); + tmpDir.mkdir(); + localLogDir.mkdir(); + remoteLogDir.mkdir(); + LOG.info("Created localDir in " + localDir.getAbsolutePath()); + LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath()); + + String bindAddress = "0.0.0.0:12345"; + conf.set(YarnConfiguration.NM_ADDRESS, bindAddress); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath()); + conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1); + // Default delSrvc + delSrvc = createDeletionService(); + delSrvc.init(conf); + exec = createContainerExecutor(); + dirsHandler = new LocalDirsHandlerService(); + nodeHealthChecker = new NodeHealthCheckerService( + NodeManager.getNodeHealthScriptRunner(conf), dirsHandler); + nodeHealthChecker.init(conf); + } @Test public void testApplicationRecovery() throws Exception { - YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true); conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true); - conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234"); conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true); conf.set(YarnConfiguration.YARN_ADMIN_ACL, "yarn_admin_user"); NMStateStoreService stateStore = new NMMemoryStateStoreService(); @@ -234,6 +290,91 @@ public void testApplicationRecovery() throws Exception { } @Test + public void testContainerResizeRecovery() throws Exception { + conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true); + conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true); + NMStateStoreService stateStore = new NMMemoryStateStoreService(); + stateStore.init(conf); + stateStore.start(); + Context context = createContext(conf, stateStore); + ContainerManagerImpl cm = createContainerManager(context, delSrvc); + cm.init(conf); + cm.start(); + // add an application by starting a container + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId attemptId = + ApplicationAttemptId.newInstance(appId, 1); + ContainerId cid = ContainerId.newContainerId(attemptId, 1); + Map containerEnv = Collections.emptyMap(); + Map serviceData = Collections.emptyMap(); + Credentials containerCreds = new Credentials(); + DataOutputBuffer dob = new DataOutputBuffer(); + containerCreds.writeTokenStorageToStream(dob); + ByteBuffer containerTokens = ByteBuffer.wrap(dob.getData(), 0, + dob.getLength()); + Map acls = Collections.emptyMap(); + File tmpDir = new File("target", + this.getClass().getSimpleName() + "-tmpDir"); + File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile"); + PrintWriter fileWriter = new PrintWriter(scriptFile); + if (Shell.WINDOWS) { + fileWriter.println("@ping -n 100 127.0.0.1 >nul"); + } else { + fileWriter.write("\numask 0"); + fileWriter.write("\nexec sleep 100"); + } + fileWriter.close(); + FileContext localFS = FileContext.getLocalFSFileContext(); + URL resource_alpha = + ConverterUtils.getYarnUrlFromPath(localFS + .makeQualified(new Path(scriptFile.getAbsolutePath()))); + LocalResource rsrc_alpha = RecordFactoryProvider + .getRecordFactory(null).newRecordInstance(LocalResource.class); + rsrc_alpha.setResource(resource_alpha); + rsrc_alpha.setSize(-1); + rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION); + rsrc_alpha.setType(LocalResourceType.FILE); + rsrc_alpha.setTimestamp(scriptFile.lastModified()); + String destinationFile = "dest_file"; + Map localResources = new HashMap<>(); + localResources.put(destinationFile, rsrc_alpha); + List commands = + Arrays.asList(Shell.getRunScriptCommand(scriptFile)); + ContainerLaunchContext clc = ContainerLaunchContext.newInstance( + localResources, containerEnv, commands, serviceData, + containerTokens, acls); + StartContainersResponse startResponse = startContainer( + context, cm, cid, clc, null); + assertTrue(startResponse.getFailedRequests().isEmpty()); + assertEquals(1, context.getApplications().size()); + Application app = context.getApplications().get(appId); + assertNotNull(app); + // make sure the container reaches RUNNING state + waitForNMContainerState(cm, cid, + org.apache.hadoop.yarn.server.nodemanager + .containermanager.container.ContainerState.RUNNING); + Resource targetResource = Resource.newInstance(2048, 2); + IncreaseContainersResourceResponse increaseResponse = + increaseContainersResource(context, cm, cid, targetResource); + assertTrue(increaseResponse.getFailedRequests().isEmpty()); + // check status + ContainerStatus containerStatus = getContainerStatus(context, cm, cid); + assertEquals(targetResource, containerStatus.getCapability()); + // restart and verify container is running and recovered + // to the correct size + cm.stop(); + context = createContext(conf, stateStore); + cm = createContainerManager(context); + cm.init(conf); + cm.start(); + assertEquals(1, context.getApplications().size()); + app = context.getApplications().get(appId); + assertNotNull(app); + containerStatus = getContainerStatus(context, cm, cid); + assertEquals(targetResource, containerStatus.getCapability()); + } + + @Test public void testContainerCleanupOnShutdown() throws Exception { ApplicationId appId = ApplicationId.newInstance(0, 1); ApplicationAttemptId attemptId = @@ -257,10 +398,8 @@ public void testContainerCleanupOnShutdown() throws Exception { LogAggregationContext.newInstance("includePattern", "excludePattern"); // verify containers are stopped on shutdown without recovery - YarnConfiguration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, false); conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, false); - conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234"); Context context = createContext(conf, new NMNullStateStoreService()); ContainerManagerImpl cm = spy(createContainerManager(context)); cm.init(conf); @@ -306,12 +445,36 @@ public void testContainerCleanupOnShutdown() throws Exception { verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class)); } - private NMContext createContext(YarnConfiguration conf, + private ContainerManagerImpl createContainerManager(Context context, + DeletionService delSrvc) { + return new ContainerManagerImpl(context, exec, delSrvc, + mock(NodeStatusUpdater.class), metrics, dirsHandler) { + @Override + public void + setBlockNewContainerRequests(boolean blockNewContainerRequests) { + // do nothing + } + @Override + protected void authorizeGetAndStopContainerRequest( + ContainerId containerId, Container container, + boolean stopRequest, NMTokenIdentifier identifier) + throws YarnException { + if(container == null || container.getUser().equals("Fail")){ + throw new YarnException("Reject this container"); + } + } + }; + } + + 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){ + public int getHttpPort() { + return HTTP_PORT; + } + }; // simulate registration with RM MasterKey masterKey = new MasterKeyPBImpl(); masterKey.setKeyId(123); @@ -349,6 +512,58 @@ public StartContainersResponse run() throws Exception { }); } + private IncreaseContainersResourceResponse increaseContainersResource( + Context context, final ContainerManagerImpl cm, ContainerId cid, + Resource capability) throws Exception { + UserGroupInformation user = UserGroupInformation.createRemoteUser( + cid.getApplicationAttemptId().toString()); + // construct container resource increase request + final List increaseTokens = new ArrayList(); + // add increase request + Token containerToken = TestContainerManager.createContainerToken( + cid, 0, context.getNodeId(), user.getShortUserName(), + capability, context.getContainerTokenSecretManager(), null); + increaseTokens.add(containerToken); + final IncreaseContainersResourceRequest increaseRequest = + IncreaseContainersResourceRequest.newInstance(increaseTokens); + NMTokenIdentifier nmToken = new NMTokenIdentifier( + cid.getApplicationAttemptId(), context.getNodeId(), + user.getShortUserName(), + context.getNMTokenSecretManager().getCurrentKey().getKeyId()); + user.addTokenIdentifier(nmToken); + return user.doAs( + new PrivilegedExceptionAction() { + @Override + public IncreaseContainersResourceResponse run() throws Exception { + return cm.increaseContainersResource(increaseRequest); + } + }); + } + + private ContainerStatus getContainerStatus( + Context context, final ContainerManagerImpl cm, ContainerId cid) + throws Exception { + UserGroupInformation user = UserGroupInformation.createRemoteUser( + cid.getApplicationAttemptId().toString()); + NMTokenIdentifier nmToken = new NMTokenIdentifier( + cid.getApplicationAttemptId(), context.getNodeId(), + user.getShortUserName(), + context.getNMTokenSecretManager().getCurrentKey().getKeyId()); + user.addTokenIdentifier(nmToken); + List containerIds = new ArrayList<>(); + containerIds.add(cid); + final GetContainerStatusesRequest gcsRequest = + GetContainerStatusesRequest.newInstance(containerIds); + return user.doAs( + new PrivilegedExceptionAction() { + @Override + public ContainerStatus run() throws Exception { + return cm.getContainerStatuses(gcsRequest) + .getContainerStatuses().get(0); + } + }); + } + private void waitForAppState(Application app, ApplicationState state) throws Exception { final int msecPerSleep = 10; 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/monitor/MockResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java new file mode 100644 index 0000000..4a18a8c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java @@ -0,0 +1,69 @@ +/** + * 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.containermanager.monitor; + +import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin; + +public class MockResourceCalculatorPlugin extends ResourceCalculatorPlugin { + + @Override + public long getVirtualMemorySize() { + return 0; + } + + @Override + public long getPhysicalMemorySize() { + return 0; + } + + @Override + public long getAvailableVirtualMemorySize() { + return 0; + } + + @Override + public long getAvailablePhysicalMemorySize() { + return 0; + } + + @Override + public int getNumProcessors() { + return 0; + } + + @Override + public int getNumCores() { + return 0; + } + + @Override + public long getCpuFrequency() { + return 0; + } + + @Override + public long getCumulativeCpuTime() { + return 0; + } + + @Override + public float getCpuUsage() { + return 0; + } +} 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/monitor/MockResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java new file mode 100644 index 0000000..c5aaa77 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.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.server.nodemanager.containermanager.monitor; + +import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; + +public class MockResourceCalculatorProcessTree extends ResourceCalculatorProcessTree { + + private long rssMemorySize = 0; + + public MockResourceCalculatorProcessTree(String root) { + super(root); + } + + @Override + public void updateProcessTree() { + } + + @Override + public String getProcessTreeDump() { + return ""; + } + + @Override + public long getCumulativeCpuTime() { + return 0; + } + + @Override + public boolean checkPidPgrpidForMatch() { + return true; + } + + public void setRssMemorySize(long rssMemorySize) { + this.rssMemorySize = rssMemorySize; + } + + public long getRssMemorySize() { + return this.rssMemorySize; + } +} 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/monitor/TestContainersMonitorResourceChange.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java new file mode 100644 index 0000000..d7f89fc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java @@ -0,0 +1,248 @@ +/** + * 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.containermanager.monitor; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; + +import org.apache.hadoop.conf.Configuration; +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.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; +import org.apache.hadoop.yarn.server.nodemanager.Context; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType; +import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ProcessTreeInfo; +import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerLivenessContext; +import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext; +import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext; +import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext; +import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +public class TestContainersMonitorResourceChange { + + private ContainersMonitorImpl containersMonitor; + private MockExecutor executor; + private Configuration conf; + private AsyncDispatcher dispatcher; + private Context context; + private MockContainerEventHandler containerEventHandler; + + private static class MockExecutor extends ContainerExecutor { + @Override + public void init() throws IOException { + } + @Override + public void startLocalizer(LocalizerStartContext ctx) + throws IOException, InterruptedException { + } + @Override + public int launchContainer(ContainerStartContext ctx) throws + IOException { + return 0; + } + @Override + public boolean signalContainer(ContainerSignalContext ctx) + throws IOException { + return true; + } + @Override + public void deleteAsUser(DeletionAsUserContext ctx) + throws IOException, InterruptedException { + } + @Override + public String getProcessId(ContainerId containerId) { + return String.valueOf(containerId.getContainerId()); + } + @Override + public boolean isContainerAlive(ContainerLivenessContext ctx) + throws IOException { + return true; + } + } + + private static class MockContainerEventHandler implements + EventHandler { + final private Set killedContainer + = new HashSet<>(); + @Override + public void handle(ContainerEvent event) { + if (event.getType() == ContainerEventType.KILL_CONTAINER) { + synchronized (killedContainer) { + killedContainer.add(event.getContainerID()); + } + } + } + public boolean isContainerKilled(ContainerId containerId) { + synchronized (killedContainer) { + return killedContainer.contains(containerId); + } + } + } + + @Before + public void setup() { + executor = new MockExecutor(); + dispatcher = new AsyncDispatcher(); + context = Mockito.mock(Context.class); + Mockito.doReturn(new ConcurrentSkipListMap()) + .when(context).getContainers(); + conf = new Configuration(); + conf.set( + YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, + MockResourceCalculatorPlugin.class.getCanonicalName()); + conf.set( + YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, + MockResourceCalculatorProcessTree.class.getCanonicalName()); + dispatcher.init(conf); + dispatcher.start(); + containerEventHandler = new MockContainerEventHandler(); + dispatcher.register(ContainerEventType.class, containerEventHandler); + } + + @After + public void tearDown() throws Exception { + if (containersMonitor != null) { + containersMonitor.stop(); + } + if (dispatcher != null) { + dispatcher.stop(); + } + } + + @Test + public void testContainersResourceChange() throws Exception { + // set container monitor interval to be 20ms + conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L); + containersMonitor = createContainersMonitor(executor, dispatcher, context); + containersMonitor.init(conf); + containersMonitor.start(); + // create container 1 + containersMonitor.handle(new ContainerStartMonitoringEvent( + getContainerId(1), 2100L, 1000L, 1, 0, 0)); + // verify that this container is properly tracked + assertNotNull(getProcessTreeInfo(getContainerId(1))); + assertEquals(1000L, getProcessTreeInfo(getContainerId(1)) + .getPmemLimit()); + assertEquals(2100L, getProcessTreeInfo(getContainerId(1)) + .getVmemLimit()); + // sleep longer than the monitor interval to make sure resource + // enforcement has started + Thread.sleep(200); + // increase pmem usage, the container should be killed + MockResourceCalculatorProcessTree mockTree = + (MockResourceCalculatorProcessTree) getProcessTreeInfo( + getContainerId(1)).getProcessTree(); + mockTree.setRssMemorySize(2500L); + // verify that this container is killed + Thread.sleep(200); + assertTrue(containerEventHandler + .isContainerKilled(getContainerId(1))); + // create container 2 + containersMonitor.handle(new ContainerStartMonitoringEvent( + getContainerId(2), 2202009L, 1048576L, 1, 0, 0)); + // verify that this container is properly tracked + assertNotNull(getProcessTreeInfo(getContainerId(2))); + assertEquals(1048576L, getProcessTreeInfo(getContainerId(2)) + .getPmemLimit()); + assertEquals(2202009L, getProcessTreeInfo(getContainerId(2)) + .getVmemLimit()); + // trigger a change resource event, check limit after change + containersMonitor.handle(new ChangeMonitoringContainerResourceEvent( + getContainerId(2), Resource.newInstance(2, 1))); + assertEquals(2097152L, getProcessTreeInfo(getContainerId(2)) + .getPmemLimit()); + assertEquals(4404019L, getProcessTreeInfo(getContainerId(2)) + .getVmemLimit()); + // sleep longer than the monitor interval to make sure resource + // enforcement has started + Thread.sleep(200); + // increase pmem usage, the container should NOT be killed + mockTree = + (MockResourceCalculatorProcessTree) getProcessTreeInfo( + getContainerId(2)).getProcessTree(); + mockTree.setRssMemorySize(2000000L); + // verify that this container is not killed + Thread.sleep(200); + assertFalse(containerEventHandler + .isContainerKilled(getContainerId(2))); + containersMonitor.stop(); + } + + @Test + public void testContainersResourceChangeIsTriggeredImmediately() + throws Exception { + // set container monitor interval to be 20s + conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20000L); + containersMonitor = createContainersMonitor(executor, dispatcher, context); + containersMonitor.init(conf); + containersMonitor.start(); + // sleep 1 second to make sure the container monitor thread is + // now waiting for the next monitor cycle + Thread.sleep(1000); + // create a container with id 3 + containersMonitor.handle(new ContainerStartMonitoringEvent( + getContainerId(3), 2202009L, 1048576L, 1, 0, 0)); + // Verify that this container has been tracked + assertNotNull(getProcessTreeInfo(getContainerId(3))); + // trigger a change resource event, check limit after change + containersMonitor.handle(new ChangeMonitoringContainerResourceEvent( + getContainerId(3), Resource.newInstance(2, 1))); + // verify that this container has been properly tracked with the + // correct size + assertEquals(2097152L, getProcessTreeInfo(getContainerId(3)) + .getPmemLimit()); + assertEquals(4404019L, getProcessTreeInfo(getContainerId(3)) + .getVmemLimit()); + containersMonitor.stop(); + } + + private ContainersMonitorImpl createContainersMonitor( + ContainerExecutor containerExecutor, AsyncDispatcher dispatcher, + Context context) { + return new ContainersMonitorImpl(containerExecutor, dispatcher, context); + } + + private ContainerId getContainerId(int id) { + return ContainerId.newContainerId(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(123456L, 1), 1), id); + } + + private ProcessTreeInfo getProcessTreeInfo(ContainerId id) { + return containersMonitor.trackingContainers.get(id); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java index 4dc4648..c0210d5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java @@ -38,7 +38,12 @@ Resource resource = Records.newRecord(Resource.class); resource.setMemory(512); //512MiB resource.setVirtualCores(2); - + Resource largerResource = Records.newRecord(Resource.class); + largerResource.setMemory(1024); + largerResource.setVirtualCores(2); + Resource smallerResource = Records.newRecord(Resource.class); + smallerResource.setMemory(256); + smallerResource.setVirtualCores(1); metrics.addResource(total); @@ -65,15 +70,20 @@ metrics.initingContainer(); metrics.runningContainer(); + // Increase resource for a container + metrics.changeContainer(resource, largerResource); + // Decrease resource for a container + metrics.changeContainer(resource, smallerResource); + Assert.assertTrue(!metrics.containerLaunchDuration.changed()); metrics.addContainerLaunchDuration(1); Assert.assertTrue(metrics.containerLaunchDuration.changed()); // availableGB is expected to be floored, // while allocatedGB is expected to be ceiled. - // allocatedGB: 3.5GB allocated memory is shown as 4GB - // availableGB: 4.5GB available memory is shown as 4GB - checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 14, 2); + // allocatedGB: 3.75GB allocated memory is shown as 4GB + // availableGB: 4.25GB available memory is shown as 4GB + checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 13, 3); } private void checkMetrics(int launched, int completed, int failed, int killed, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java index e0487e7..a1c95ab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto; import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto; @@ -122,9 +123,10 @@ public synchronized void removeApplication(ApplicationId appId) rcsCopy.killed = rcs.killed; rcsCopy.diagnostics = rcs.diagnostics; rcsCopy.startRequest = rcs.startRequest; + rcsCopy.capability = rcs.capability; result.add(rcsCopy); } - return new ArrayList(); + return result; } @Override @@ -153,6 +155,13 @@ public synchronized void storeContainerLaunched(ContainerId containerId) } @Override + public synchronized void storeContainerResourceChanged( + ContainerId containerId, Resource capability) throws IOException { + RecoveredContainerState rcs = getRecoveredContainerState(containerId); + rcs.capability = capability; + } + + @Override public synchronized void storeContainerKilled(ContainerId containerId) throws IOException { RecoveredContainerState rcs = getRecoveredContainerState(containerId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java index 1804424..08b49e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java @@ -298,6 +298,17 @@ public void testContainerStorage() throws IOException { assertEquals(containerReq, rcs.getStartRequest()); assertEquals(diags.toString(), rcs.getDiagnostics()); + // increase the container size, and verify recovered + stateStore.storeContainerResourceChanged(containerId, Resource.newInstance(2468, 4)); + restartStateStore(); + recoveredContainers = stateStore.loadContainersState(); + assertEquals(1, recoveredContainers.size()); + rcs = recoveredContainers.get(0); + assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus()); + assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode()); + assertEquals(false, rcs.getKilled()); + assertEquals(Resource.newInstance(2468, 4), rcs.getCapability()); + // mark the container killed, add some more diags, and verify recovered diags.append("some more diags for container"); stateStore.storeContainerDiagnostics(containerId, diags); 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/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java index b2ccb61..394a92c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java @@ -132,6 +132,10 @@ public Resource getResource() { } @Override + public void setResource(Resource targetResource) { + } + + @Override public ContainerTokenIdentifier getContainerTokenIdentifier() { return this.containerTokenIdentifier; } 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 14142dee..87c7bfa 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 @@ -451,11 +451,13 @@ public AllocateResponse allocate(AllocateRequest request) req.setNodeLabelExpression(asc.getNodeLabelExpression()); } } + + Resource maximumCapacity = rScheduler.getMaximumResourceCapability(); // sanity check try { RMServerUtils.normalizeAndValidateRequests(ask, - rScheduler.getMaximumResourceCapability(), app.getQueue(), + maximumCapacity, app.getQueue(), rScheduler, rmContext); } catch (InvalidResourceRequestException e) { LOG.warn("Invalid resource ask by application " + appAttemptId, e); @@ -469,6 +471,15 @@ public AllocateResponse allocate(AllocateRequest request) throw e; } + try { + RMServerUtils.increaseDecreaseRequestSanityCheck(rmContext, + request.getIncreaseRequests(), request.getDecreaseRequests(), + maximumCapacity); + } catch (InvalidResourceRequestException e) { + LOG.warn(e); + throw e; + } + // In the case of work-preserving AM restart, it's possible for the // AM to release containers from the earlier attempt. if (!app.getApplicationSubmissionContext() @@ -493,8 +504,9 @@ public AllocateResponse allocate(AllocateRequest request) allocation = EMPTY_ALLOCATION; } else { allocation = - this.rScheduler.allocate(appAttemptId, ask, release, - blacklistAdditions, blacklistRemovals); + this.rScheduler.allocate(appAttemptId, ask, release, + blacklistAdditions, blacklistRemovals, + request.getIncreaseRequests(), request.getDecreaseRequests()); } if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) { @@ -540,6 +552,10 @@ public AllocateResponse allocate(AllocateRequest request) .pullJustFinishedContainers()); allocateResponse.setResponseId(lastResponse.getResponseId() + 1); allocateResponse.setAvailableResources(allocation.getResourceLimit()); + + // Handling increased/decreased containers + allocateResponse.setIncreasedContainers(allocation.getIncreasedContainers()); + allocateResponse.setDecreasedContainers(allocation.getDecreasedContainers()); allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java index f049d97..cd9a61d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java @@ -56,6 +56,8 @@ public static final String RELEASE_CONTAINER = "AM Released Container"; public static final String UPDATE_APP_PRIORITY = "Update Application Priority Request"; + public static final String CHANGE_CONTAINER_RESOURCE = + "AM Changed Container Resource"; // Some commonly used descriptions public static final String UNAUTHORIZED_USER = "Unauthorized user"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java index 4d2e41c..cc30593 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java @@ -22,8 +22,10 @@ import java.util.ArrayList; import java.util.EnumSet; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; @@ -34,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.api.records.Resource; @@ -49,10 +52,14 @@ import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; 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; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; /** @@ -107,6 +114,89 @@ public static void normalizeAndValidateRequests(List ask, queueName, scheduler, rmContext, queueInfo); } } + + /** + * Normalize container increase/decrease request, it will normalize and update + * ContainerResourceChangeRequest.targetResource + * + *
    +   * - Throw exception when any other error happens
    +   * 
    + */ + public static void checkAndNormalizeContainerChangeRequest( + RMContext rmContext, ContainerResourceChangeRequest request, + boolean increase) throws InvalidResourceRequestException { + ContainerId containerId = request.getContainerId(); + ResourceScheduler scheduler = rmContext.getScheduler(); + RMContainer rmContainer = scheduler.getRMContainer(containerId); + ResourceCalculator rc = scheduler.getResourceCalculator(); + + if (null == rmContainer) { + String msg = + "Failed to get rmContainer for " + + (increase ? "increase" : "decrease") + + " request, with container-id=" + containerId; + throw new InvalidResourceRequestException(msg); + } + + if (rmContainer.getState() != RMContainerState.RUNNING) { + String msg = + "rmContainer's state is not RUNNING, for " + + (increase ? "increase" : "decrease") + + " request, with container-id=" + containerId; + throw new InvalidResourceRequestException(msg); + } + + Resource targetResource = Resources.normalize(rc, request.getCapability(), + scheduler.getMinimumResourceCapability(), + scheduler.getMaximumResourceCapability(), + scheduler.getMinimumResourceCapability()); + + // Compare targetResource and original resource + Resource originalResource = rmContainer.getAllocatedResource(); + + // Resource comparasion should be >= (or <=) for all resource vectors, for + // example, you cannot request target resource of a <10G, 10> container to + // <20G, 8> + if (increase) { + if (originalResource.getMemory() > targetResource.getMemory() + || originalResource.getVirtualCores() > targetResource + .getVirtualCores()) { + String msg = + "Trying to increase a container, but target resource has some" + + " resource < original resource, target=" + targetResource + + " original=" + originalResource + " containerId=" + + containerId; + throw new InvalidResourceRequestException(msg); + } + } else { + if (originalResource.getMemory() < targetResource.getMemory() + || originalResource.getVirtualCores() < targetResource + .getVirtualCores()) { + String msg = + "Trying to decrease a container, but target resource has " + + "some resource > original resource, target=" + targetResource + + " original=" + originalResource + " containerId=" + + containerId; + throw new InvalidResourceRequestException(msg); + } + } + + RMNode rmNode = rmContext.getRMNodes().get(rmContainer.getAllocatedNode()); + + // Target resource of the increase request is more than NM can offer + if (!Resources.fitsIn(scheduler.getResourceCalculator(), + scheduler.getClusterResource(), targetResource, + rmNode.getTotalCapability())) { + String msg = "Target resource=" + targetResource + " of containerId=" + + containerId + " is more than node's total resource=" + + rmNode.getTotalCapability(); + throw new InvalidResourceRequestException(msg); + } + + // Update normalized target resource + request.setCapability(targetResource); + } /* * @throw InvalidResourceBlacklistRequestException if the @@ -123,6 +213,80 @@ public static void validateBlacklistRequest( } } } + + /** + * Check if we have: + * - Request for same containerId and different target resource + * - If targetResources violates maximum/minimumAllocation + */ + public static void increaseDecreaseRequestSanityCheck(RMContext rmContext, + List incRequests, + List decRequests, + Resource maximumAllocation) throws InvalidResourceRequestException { + checkDuplicatedIncreaseDecreaseRequest(incRequests, decRequests); + validateIncreaseDecreaseRequest(rmContext, incRequests, maximumAllocation, + true); + validateIncreaseDecreaseRequest(rmContext, decRequests, maximumAllocation, + false); + } + + private static void checkDuplicatedIncreaseDecreaseRequest( + List incRequests, + List decRequests) + throws InvalidResourceRequestException { + String msg = "There're multiple increase or decrease container requests " + + "for same containerId="; + Set existedContainerIds = new HashSet(); + if (incRequests != null) { + for (ContainerResourceChangeRequest r : incRequests) { + if (!existedContainerIds.add(r.getContainerId())) { + throw new InvalidResourceRequestException(msg + r.getContainerId()); + } + } + } + + if (decRequests != null) { + for (ContainerResourceChangeRequest r : decRequests) { + if (!existedContainerIds.add(r.getContainerId())) { + throw new InvalidResourceRequestException(msg + r.getContainerId()); + } + } + } + } + + private static void validateIncreaseDecreaseRequest(RMContext rmContext, + List requests, Resource maximumAllocation, + boolean increase) + throws InvalidResourceRequestException { + if (requests == null) { + return; + } + for (ContainerResourceChangeRequest request : requests) { + if (request.getCapability().getMemory() < 0 + || request.getCapability().getMemory() > maximumAllocation + .getMemory()) { + throw new InvalidResourceRequestException("Invalid " + + (increase ? "increase" : "decrease") + " request" + + ", requested memory < 0" + + ", or requested memory > max configured" + ", requestedMemory=" + + request.getCapability().getMemory() + ", maxMemory=" + + maximumAllocation.getMemory()); + } + if (request.getCapability().getVirtualCores() < 0 + || request.getCapability().getVirtualCores() > maximumAllocation + .getVirtualCores()) { + throw new InvalidResourceRequestException("Invalid " + + (increase ? "increase" : "decrease") + " request" + + ", requested virtual cores < 0" + + ", or requested virtual cores > max configured" + + ", requestedVirtualCores=" + + request.getCapability().getVirtualCores() + ", maxVirtualCores=" + + maximumAllocation.getVirtualCores()); + } + + checkAndNormalizeContainerChangeRequest(rmContext, request, increase); + } + } /** * It will validate to make sure all the containers belong to correct diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java index 100e991..557f6d4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java @@ -449,6 +449,8 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) getResponseId() + 1, NodeAction.NORMAL, null, null, null, null, nextHeartBeatInterval); rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse); + rmNode.updateNodeHeartbeatResponseForContainersDecreasing( + nodeHeartBeatResponse); populateKeys(request, nodeHeartBeatResponse); @@ -461,8 +463,9 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request) // 4. Send status to RMNode, saving the latest response. RMNodeStatusEvent nodeStatusEvent = new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(), - remoteNodeStatus.getContainersStatuses(), - remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse); + remoteNodeStatus.getContainersStatuses(), + remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse, + remoteNodeStatus.getIncreasedContainers()); if (request.getLogAggregationReportsForApps() != null && !request.getLogAggregationReportsForApps().isEmpty()) { nodeStatusEvent.setLogAggregationReportsForApps(request diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index 629b2a3..43de3ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -971,7 +971,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, Collections.singletonList(appAttempt.amReq), EMPTY_CONTAINER_RELEASE_LIST, amBlacklist.getAdditions(), - amBlacklist.getRemovals()); + amBlacklist.getRemovals(), null, null); if (amContainerAllocation != null && amContainerAllocation.getContainers() != null) { assert (amContainerAllocation.getContainers().size() == 0); @@ -995,7 +995,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, Allocation amContainerAllocation = appAttempt.scheduler.allocate(appAttempt.applicationAttemptId, EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null, - null); + null, null, null); // There must be at least one container allocated, because a // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed, // and is put in SchedulerApplication#newlyAllocatedContainers. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index 21d79ee..dc0d9ba 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -82,4 +82,8 @@ String getNodeHttpAddress(); String getNodeLabelExpression(); + + boolean hasIncreaseReservation(); + + void cancelIncreaseReservation(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java new file mode 100644 index 0000000..920cfdb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; + +public class RMContainerChangeResourceEvent extends RMContainerEvent { + + final Resource targetResource; + final boolean increase; + + public RMContainerChangeResourceEvent(ContainerId containerId, + Resource targetResource, boolean increase) { + super(containerId, RMContainerEventType.CHANGE_RESOURCE); + + this.targetResource = targetResource; + this.increase = increase; + } + + public Resource getTargetResource() { + return targetResource; + } + + public boolean isIncrease() { + return increase; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java index 259d68b3..a3b4b76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java @@ -25,6 +25,10 @@ ACQUIRED, KILL, // Also from Node on NodeRemoval RESERVED, + + // when a container acquired by AM after + // it increased/decreased + ACQUIRE_UPDATED_CONTAINER, LAUNCHED, FINISHED, @@ -35,5 +39,12 @@ // Source: ContainerAllocationExpirer EXPIRE, - RECOVER + RECOVER, + + // Source: Scheduler + // Resource change approved by scheduler + CHANGE_RESOURCE, + + // NM reported resource change is done + NM_DONE_CHANGE_RESOURCE } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index a3d8bee..8133657 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -118,7 +118,18 @@ .addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED, RMContainerEventType.RELEASED, new KillTransition()) .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING, - RMContainerEventType.EXPIRE) + RMContainerEventType.RESERVED, new ContainerReservedTransition()) + .addTransition(RMContainerState.RUNNING, RMContainerState.EXPIRED, + RMContainerEventType.EXPIRE, + new ContainerExpiredWhileRunningTransition()) + .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING, + RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition()) + .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING, + RMContainerEventType.ACQUIRE_UPDATED_CONTAINER, + new ContainerAcquiredWhileRunningTransition()) + .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING, + RMContainerEventType.NM_DONE_CHANGE_RESOURCE, + new NMReportedContainerChangeIsDoneTransition()) // Transitions from COMPLETED state .addTransition(RMContainerState.COMPLETED, RMContainerState.COMPLETED, @@ -140,9 +151,7 @@ RMContainerEventType.KILL, RMContainerEventType.FINISHED)) // create the topology tables - .installTopology(); - - + .installTopology(); private final StateMachine stateMachine; @@ -166,6 +175,8 @@ private ContainerStatus finishedStatus; private boolean isAMContainer; private List resourceRequests; + + private volatile boolean hasIncreaseReservation = false; public RMContainerImpl(Container container, ApplicationAttemptId appAttemptId, NodeId nodeId, String user, @@ -264,7 +275,12 @@ public Priority getReservedPriority() { @Override public Resource getAllocatedResource() { - return container.getResource(); + try { + readLock.lock(); + return container.getResource(); + } finally { + readLock.unlock(); + } } @Override @@ -471,8 +487,8 @@ public RMContainerState transition(RMContainerImpl container, } } - private static final class ContainerReservedTransition extends - BaseTransition { + private static final class ContainerReservedTransition + extends BaseTransition { @Override public void transition(RMContainerImpl container, RMContainerEvent event) { @@ -480,6 +496,12 @@ public void transition(RMContainerImpl container, RMContainerEvent event) { container.reservedResource = e.getReservedResource(); container.reservedNode = e.getReservedNode(); container.reservedPriority = e.getReservedPriority(); + + if (!EnumSet.of(RMContainerState.NEW, RMContainerState.RESERVED) + .contains(container.getState())) { + // When container's state != NEW/RESERVED, it is an increase reservation + container.hasIncreaseReservation = true; + } } } @@ -509,6 +531,70 @@ public void transition(RMContainerImpl container, RMContainerEvent event) { .getApplicationAttemptId().getApplicationId(), container.nodeId)); } } + + private static final class ContainerAcquiredWhileRunningTransition extends + BaseTransition { + + @Override + public void transition(RMContainerImpl container, RMContainerEvent event) { + RMContainerUpdatesAcquiredEvent acquiredEvent = + (RMContainerUpdatesAcquiredEvent) event; + if (acquiredEvent.isIncreasedContainer()) { + // If container is increased but not acquired by AM, we will start + // containerAllocationExpirer for this container in this transition. + container.containerAllocationExpirer.register(event.getContainerId()); + } + } + } + + private static final class NMReportedContainerChangeIsDoneTransition + extends BaseTransition { + + @Override + public void transition(RMContainerImpl container, RMContainerEvent event) { + // Unregister the allocation expirer, it is already increased.. + container.containerAllocationExpirer.unregister(event.getContainerId()); + } + } + + private static final class ContainerExpiredWhileRunningTransition extends + BaseTransition { + + @Override + public void transition(RMContainerImpl container, RMContainerEvent event) { + // When the container expired, and it has a pending increased request, we + // will kill the container. + // TODO, we can do better for this: roll back container resource to the + // resource before increase, and notify scheduler about this decrease as + // well. Will do that in a separated JIRA. + new KillTransition().transition(container, event); + } + } + + private static final class ChangeResourceTransition extends BaseTransition { + + @Override + public void transition(RMContainerImpl container, RMContainerEvent event) { + RMContainerChangeResourceEvent changeEvent = (RMContainerChangeResourceEvent)event; + + // Register with containerAllocationExpirer. + // For now, we assume timeout for increase is as same as container + // allocation. + if (!changeEvent.isIncrease()) { + // if this is a decrease request, if container was increased but not + // told to NM, we can consider previous increase is cancelled, + // unregister from the containerAllocationExpirer + container.containerAllocationExpirer.unregister(container + .getContainerId()); + } + + container.container.setResource(changeEvent.getTargetResource()); + + // We reach here means we either allocated increase reservation OR + // decreased container, reservation will be cancelled anyway. + container.hasIncreaseReservation = false; + } + } private static final class ContainerRescheduledTransition extends FinishedTransition { @@ -561,13 +647,14 @@ private static void updateAttemptMetrics(RMContainerImpl container) { RMAppAttempt rmAttempt = container.rmContext.getRMApps() .get(container.getApplicationAttemptId().getApplicationId()) .getCurrentAppAttempt(); - if (ContainerExitStatus.PREEMPTED == container.finishedStatus - .getExitStatus()) { - rmAttempt.getRMAppAttemptMetrics().updatePreemptionInfo(resource, - container); - } if (rmAttempt != null) { + if (ContainerExitStatus.PREEMPTED == container.finishedStatus + .getExitStatus()) { + rmAttempt.getRMAppAttemptMetrics().updatePreemptionInfo(resource, + container); + } + long usedMillis = container.finishTime - container.creationTime; long memorySeconds = resource.getMemory() * usedMillis / DateUtils.MILLIS_PER_SECOND; @@ -665,4 +752,14 @@ public int compareTo(RMContainer o) { } return -1; } + + @Override + public boolean hasIncreaseReservation() { + return hasIncreaseReservation; + } + + @Override + public void cancelIncreaseReservation() { + hasIncreaseReservation = false; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java new file mode 100644 index 0000000..0dccc5f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java @@ -0,0 +1,35 @@ +/** + * 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.rmcontainer; + +import org.apache.hadoop.yarn.api.records.ContainerId; + +public class RMContainerUpdatesAcquiredEvent extends RMContainerEvent { + private final boolean increasedContainer; + + public RMContainerUpdatesAcquiredEvent(ContainerId containerId, + boolean increasedContainer) { + super(containerId, RMContainerEventType.ACQUIRE_UPDATED_CONTAINER); + this.increasedContainer = increasedContainer; + } + + public boolean isIncreasedContainer() { + return increasedContainer; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java index 6bb0971..f28422a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java @@ -24,6 +24,7 @@ import org.apache.hadoop.net.Node; 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.NodeId; import org.apache.hadoop.yarn.api.records.NodeState; @@ -146,4 +147,12 @@ * @return labels in this node */ public Set getNodeLabels(); + + /** + * Update containers to be decreased + */ + public void updateNodeHeartbeatResponseForContainersDecreasing( + NodeHeartbeatResponse response); + + public List pullNewlyIncreasedContainers(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java new file mode 100644 index 0000000..62925ad --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.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.server.resourcemanager.rmnode; + +import java.util.List; + +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.NodeId; + +public class RMNodeDecreaseContainerEvent extends RMNodeEvent { + final List toBeDecreasedContainers; + + public RMNodeDecreaseContainerEvent(NodeId nodeId, + List toBeDecreasedContainers) { + super(nodeId, RMNodeEventType.DECREASE_CONTAINER); + + this.toBeDecreasedContainers = toBeDecreasedContainers; + } + + public List getToBeDecreasedContainers() { + return toBeDecreasedContainers; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java index 27ba1c0..a68c894 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java @@ -42,6 +42,7 @@ // Source: Container CONTAINER_ALLOCATED, CLEANUP_CONTAINER, + DECREASE_CONTAINER, // Source: RMAppAttempt FINISHED_CONTAINERS_PULLED_BY_AM, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java index 7a1ba74..7a43598 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java @@ -19,9 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ConcurrentLinkedQueue; @@ -36,6 +40,7 @@ import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; 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.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; @@ -131,6 +136,12 @@ /* the list of applications that are running on this node */ private final List runningApplications = new ArrayList(); + + private final Map toBeDecreasedContainers = + new HashMap<>(); + + private final Map nmReportedIncreasedContainers = + new HashMap<>(); private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory .newRecordInstance(NodeHeartbeatResponse.class); @@ -178,6 +189,9 @@ .addTransition(NodeState.RUNNING, NodeState.SHUTDOWN, RMNodeEventType.SHUTDOWN, new DeactivateNodeTransition(NodeState.SHUTDOWN)) + .addTransition(NodeState.RUNNING, NodeState.RUNNING, + RMNodeEventType.DECREASE_CONTAINER, + new DecreaseContainersTransition()) //Transitions from REBOOTED state .addTransition(NodeState.REBOOTED, NodeState.REBOOTED, @@ -430,6 +444,24 @@ public void updateNodeHeartbeatResponseForCleanup(NodeHeartbeatResponse response this.writeLock.unlock(); } }; + + @VisibleForTesting + public Collection getToBeDecreasedContainers() { + return toBeDecreasedContainers.values(); + } + + @Override + public void updateNodeHeartbeatResponseForContainersDecreasing( + NodeHeartbeatResponse response) { + this.writeLock.lock(); + + try { + response.addAllContainersToDecrease(toBeDecreasedContainers.values()); + toBeDecreasedContainers.clear(); + } finally { + this.writeLock.unlock(); + } + } @Override public NodeHeartbeatResponse getLastNodeHeartBeatResponse() { @@ -759,6 +791,19 @@ public void transition(RMNodeImpl rmNode, RMNodeEvent event) { RMNodeFinishedContainersPulledByAMEvent) event).getContainers()); } } + + public static class DecreaseContainersTransition + implements SingleArcTransition { + + @Override + public void transition(RMNodeImpl rmNode, RMNodeEvent event) { + RMNodeDecreaseContainerEvent de = (RMNodeDecreaseContainerEvent) event; + + for (Container c : de.getToBeDecreasedContainers()) { + rmNode.toBeDecreasedContainers.put(c.getId(), c); + } + } + } public static class DeactivateNodeTransition implements SingleArcTransition { @@ -827,6 +872,8 @@ public NodeState transition(RMNodeImpl rmNode, RMNodeEvent event) { } rmNode.handleContainerStatus(statusEvent.getContainers()); + rmNode.handleReportedIncreasedContainers( + statusEvent.getNMReportedIncreasedContainers()); List logAggregationReportsForApps = statusEvent.getLogAggregationReportsForApps(); @@ -919,6 +966,34 @@ public int getQueueSize() { } return nlm.getLabelsOnNode(nodeId); } + + private void handleReportedIncreasedContainers( + List reportedIncreasedContainers) { + for (Container container : reportedIncreasedContainers) { + ContainerId containerId = container.getId(); + + // Don't bother with containers already scheduled for cleanup, or for + // applications already killed. The scheduler doens't need to know any + // more about this container + if (containersToClean.contains(containerId)) { + LOG.info("Container " + containerId + " already scheduled for " + + "cleanup, no further processing"); + continue; + } + + ApplicationId containerAppId = + containerId.getApplicationAttemptId().getApplicationId(); + + if (finishedApplications.contains(containerAppId)) { + LOG.info("Container " + containerId + + " belongs to an application that is already killed," + + " no further processing"); + continue; + } + + this.nmReportedIncreasedContainers.put(containerId, container); + } + } private void handleContainerStatus(List containerStatuses) { // Filter the map to only obtain just launched containers and finished @@ -989,4 +1064,22 @@ private void handleLogAggregationStatus( } } + @Override + public List pullNewlyIncreasedContainers() { + try { + writeLock.lock(); + + if (nmReportedIncreasedContainers.isEmpty()) { + return Collections.EMPTY_LIST; + } else { + List container = + new ArrayList(nmReportedIncreasedContainers.values()); + nmReportedIncreasedContainers.clear(); + return container; + } + + } finally { + writeLock.unlock(); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java index b95d7d3..8323f3c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java @@ -18,8 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode; +import java.util.Collections; import java.util.List; + import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; @@ -33,28 +36,36 @@ private final NodeHeartbeatResponse latestResponse; private final List keepAliveAppIds; private List logAggregationReportsForApps; - + private final List nmReportedIncreasedContainers; + + // Used by tests public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus, List collection, List keepAliveAppIds, NodeHeartbeatResponse latestResponse) { - super(nodeId, RMNodeEventType.STATUS_UPDATE); - this.nodeHealthStatus = nodeHealthStatus; - this.containersCollection = collection; - this.keepAliveAppIds = keepAliveAppIds; - this.latestResponse = latestResponse; - this.logAggregationReportsForApps = null; + this(nodeId, nodeHealthStatus, collection, keepAliveAppIds, + latestResponse, null); } public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus, List collection, List keepAliveAppIds, NodeHeartbeatResponse latestResponse, - List logAggregationReportsForApps) { + List nmReportedIncreasedContainers) { + this(nodeId, nodeHealthStatus, collection, keepAliveAppIds, latestResponse, + null, nmReportedIncreasedContainers); + } + + public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus, + List collection, List keepAliveAppIds, + NodeHeartbeatResponse latestResponse, + List logAggregationReportsForApps, + List nmReportedIncreasedContainers) { super(nodeId, RMNodeEventType.STATUS_UPDATE); this.nodeHealthStatus = nodeHealthStatus; this.containersCollection = collection; this.keepAliveAppIds = keepAliveAppIds; this.latestResponse = latestResponse; this.logAggregationReportsForApps = logAggregationReportsForApps; + this.nmReportedIncreasedContainers = nmReportedIncreasedContainers; } public NodeHealthStatus getNodeHealthStatus() { @@ -81,4 +92,9 @@ public void setLogAggregationReportsForApps( List logAggregationReportsForApps) { this.logAggregationReportsForApps = logAggregationReportsForApps; } + + public List getNMReportedIncreasedContainers() { + return nmReportedIncreasedContainers == null ? Collections.EMPTY_LIST + : nmReportedIncreasedContainers; + } } \ No newline at end of file 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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java index 0b6b8ef..ad28493 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java @@ -19,7 +19,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Timer; +import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -37,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -51,6 +61,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; @@ -58,13 +69,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerRecoverEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.util.resource.Resources; import com.google.common.annotations.VisibleForTesting; @@ -87,7 +100,7 @@ protected Resource clusterResource = Resource.newInstance(0, 0); protected Resource minimumAllocation; - private Resource maximumAllocation; + protected Resource maximumAllocation; private Resource configuredMaximumAllocation; private int maxNodeMemory = -1; private int maxNodeVCores = -1; @@ -231,6 +244,55 @@ protected synchronized void containerLaunchedOnNode( application.containerLaunchedOnNode(containerId, node.getNodeID()); } + + protected synchronized void containerIncreasedOnNode(ContainerId containerId, + SchedulerNode node, Container increasedContainerReportedByNM) { + // Get the application for the finished container + SchedulerApplicationAttempt application = + getCurrentAttemptForContainer(containerId); + if (application == null) { + LOG.info("Unknown application " + + containerId.getApplicationAttemptId().getApplicationId() + + " increased container " + containerId + " on node: " + node); + this.rmContext.getDispatcher().getEventHandler() + .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); + return; + } + + RMContainer rmContainer = getRMContainer(containerId); + Resource rmContainerResource = rmContainer.getAllocatedResource(); + Resource nmContainerResource = increasedContainerReportedByNM.getResource(); + + + if (Resources.equals(nmContainerResource, rmContainerResource)){ + // NM reported expected container size, tell RMContainer. Which will stop + // container expire monitor + rmContainer.handle(new RMContainerEvent(containerId, + RMContainerEventType.NM_DONE_CHANGE_RESOURCE)); + } else if (Resources.fitsIn(getResourceCalculator(), clusterResource, + nmContainerResource, rmContainerResource)) { + // when rmContainerResource >= nmContainerResource, we won't do anything, + // it is possible a container increased is issued by RM, but AM hasn't + // told NM. + } else if (Resources.fitsIn(getResourceCalculator(), clusterResource, + rmContainerResource, nmContainerResource)) { + // When rmContainerResource <= nmContainerResource, it could happen when a + // container decreased by RM before it is increased in NM. + + // Tell NM to decrease the container + this.rmContext.getDispatcher().getEventHandler() + .handle(new RMNodeDecreaseContainerEvent(node.getNodeID(), + Arrays.asList(rmContainer.getContainer()))); + } else { + // Something wrong happened, kill the container + LOG.warn("Something wrong happened, container size reported by NM" + + " is not expected, ContainerID=" + containerId + + " rm-size-resource:" + rmContainerResource + " nm-size-reosurce:" + + nmContainerResource); + this.rmContext.getDispatcher().getEventHandler() + .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); + } + } public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) { SchedulerApplication app = @@ -511,6 +573,36 @@ protected void releaseContainers(List containers, SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED); } } + + protected void decreaseContainers( + List decreaseRequests, + SchedulerApplicationAttempt attempt) { + for (SchedContainerChangeRequest request : decreaseRequests) { + if (LOG.isDebugEnabled()) { + LOG.debug("Processing decrease request:" + request); + } + + boolean hasIncreaseRequest = + attempt.removeIncreaseRequest(request.getNodeId(), + request.getPriority(), request.getContainerId()); + + if (hasIncreaseRequest) { + if (LOG.isDebugEnabled()) { + LOG.debug("While processing decrease request, found a increase request " + + "for the same container " + + request.getContainerId() + + ", removed the increase request"); + } + } + + // handle decrease request + decreaseContainer(request, attempt); + } + } + + protected abstract void decreaseContainer( + SchedContainerChangeRequest decreaseRequest, + SchedulerApplicationAttempt attempt); public SchedulerNode getSchedulerNode(NodeId nodeId) { return nodes.get(nodeId); @@ -733,4 +825,56 @@ public synchronized void setClusterMaxPriority(Configuration conf) LOG.info("Updated the cluste max priority to maxClusterLevelAppPriority = " + maxClusterLevelAppPriority); } + + /** + * Normalize container increase/decrease request, and return + * SchedulerContainerResourceChangeRequest according to given + * ContainerResourceChangeRequest. + * + *
    +   * - Returns non-null value means validation succeeded
    +   * - Throw exception when any other error happens
    +   * 
    + */ + private SchedContainerChangeRequest + checkAndNormalizeContainerChangeRequest( + ContainerResourceChangeRequest request, boolean increase) + throws YarnException { + // We have done a check in ApplicationMasterService, but RMContainer status + // / Node resource could change since AMS won't acquire lock of scheduler. + RMServerUtils.checkAndNormalizeContainerChangeRequest(rmContext, request, + increase); + ContainerId containerId = request.getContainerId(); + RMContainer rmContainer = getRMContainer(containerId); + SchedulerNode schedulerNode = + getSchedulerNode(rmContainer.getAllocatedNode()); + + return new SchedContainerChangeRequest(schedulerNode, rmContainer, + request.getCapability()); + } + + protected List + checkAndNormalizeContainerChangeRequests( + List changeRequests, + boolean increase) { + if (null == changeRequests || changeRequests.isEmpty()) { + return Collections.EMPTY_LIST; + } + + List schedulerChangeRequests = + new ArrayList(); + for (ContainerResourceChangeRequest r : changeRequests) { + SchedContainerChangeRequest sr = null; + try { + sr = checkAndNormalizeContainerChangeRequest(r, increase); + } catch (YarnException e) { + LOG.warn("Error happens when checking increase request, Ignoring.." + + " exception=", e); + continue; + } + schedulerChangeRequests.add(sr); + } + + return schedulerChangeRequests; + } } 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/Allocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java index 3f2d8af..af6caad 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java @@ -34,6 +34,9 @@ final Set fungibleContainers; final List fungibleResources; final List nmTokens; + final List increasedContainers; + final List decreasedContainers; + public Allocation(List containers, Resource resourceLimit, Set strictContainers, Set fungibleContainers, @@ -45,12 +48,22 @@ public Allocation(List containers, Resource resourceLimit, public Allocation(List containers, Resource resourceLimit, Set strictContainers, Set fungibleContainers, List fungibleResources, List nmTokens) { + this(containers, resourceLimit,strictContainers, fungibleContainers, + fungibleResources, nmTokens, null, null); + } + + public Allocation(List containers, Resource resourceLimit, + Set strictContainers, Set fungibleContainers, + List fungibleResources, List nmTokens, + List increasedContainers, List decreasedContainer) { this.containers = containers; this.resourceLimit = resourceLimit; this.strictContainers = strictContainers; this.fungibleContainers = fungibleContainers; this.fungibleResources = fungibleResources; this.nmTokens = nmTokens; + this.increasedContainers = increasedContainers; + this.decreasedContainers = decreasedContainer; } public List getContainers() { @@ -76,5 +89,12 @@ public Resource getResourceLimit() { public List getNMTokens() { return nmTokens; } - + + public List getIncreasedContainers() { + return increasedContainers; + } + + public List getDecreasedContainers() { + return decreasedContainers; + } } 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 e318d47..7623da0 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 @@ -20,10 +20,12 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; @@ -35,6 +37,8 @@ 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.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; @@ -63,8 +67,11 @@ final Set priorities = new TreeSet( new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); - final Map> requests = - new ConcurrentHashMap>(); + final Map> resourceRequestMap = + new ConcurrentHashMap>(); + final Map>> increaseRequestMap = + new ConcurrentHashMap<>(); private Set userBlacklist = new HashSet<>(); private Set amBlacklist = new HashSet<>(); @@ -114,13 +121,177 @@ public synchronized boolean isPending() { */ private synchronized void clearRequests() { priorities.clear(); - requests.clear(); + resourceRequestMap.clear(); LOG.info("Application " + applicationId + " requests cleared"); } public long getNewContainerId() { return this.containerIdCounter.incrementAndGet(); } + + public boolean hasIncreaseRequest(NodeId nodeId) { + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + return false; + } + return requestsOnNode.size() > 0; + } + + public Map + getIncreaseRequests(NodeId nodeId, Priority priority) { + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + return null; + } + + return requestsOnNode.get(priority); + } + + public synchronized boolean updateIncreaseRequests( + List increaseRequests) { + boolean resourceUpdated = false; + + for (SchedContainerChangeRequest r : increaseRequests) { + NodeId nodeId = r.getRMContainer().getAllocatedNode(); + + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + requestsOnNode = new TreeMap<>(); + increaseRequestMap.put(nodeId, requestsOnNode); + } + + SchedContainerChangeRequest prevChangeRequest = + getIncreaseRequest(nodeId, r.getPriority(), r.getContainerId()); + if (null != prevChangeRequest) { + if (Resources.equals(prevChangeRequest.getTargetCapacity(), + r.getTargetCapacity())) { + // New target capacity is as same as what we have, just ignore the new + // one + continue; + } + + // remove the old one + removeIncreaseRequest(nodeId, prevChangeRequest.getPriority(), + prevChangeRequest.getContainerId()); + } + + if (Resources.equals(r.getTargetCapacity(), r.getRMContainer().getAllocatedResource())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to increase/decrease container, " + + "target capacity = previous capacity = " + prevChangeRequest + + " for container=" + r.getContainerId() + + ". Will ignore this increase request"); + } + continue; + } + + // add the new one + resourceUpdated = true; + insertIncreaseRequest(r); + } + return resourceUpdated; + } + + // insert increase request and add missing hierarchy if missing + private void insertIncreaseRequest(SchedContainerChangeRequest request) { + NodeId nodeId = request.getNodeId(); + Priority priority = request.getPriority(); + ContainerId containerId = request.getContainerId(); + + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + requestsOnNode = + new HashMap>(); + increaseRequestMap.put(nodeId, requestsOnNode); + } + + Map requestsOnNodeWithPriority = + requestsOnNode.get(priority); + if (null == requestsOnNodeWithPriority) { + requestsOnNodeWithPriority = + new TreeMap(); + requestsOnNode.put(priority, requestsOnNodeWithPriority); + } + + requestsOnNodeWithPriority.put(containerId, request); + + // update resources + String partition = request.getRMContainer().getNodeLabelExpression(); + Resource delta = request.getDeltaCapacity(); + appResourceUsage.incPending(partition, delta); + queue.incPendingResource(partition, delta); + + if (LOG.isDebugEnabled()) { + LOG.debug("Added increase request:" + request.getContainerId() + + " delta=" + request.getDeltaCapacity()); + } + + // update priorities + priorities.add(priority); + } + + public synchronized boolean removeIncreaseRequest(NodeId nodeId, Priority priority, + ContainerId containerId) { + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + return false; + } + + Map requestsOnNodeWithPriority = + requestsOnNode.get(priority); + if (null == requestsOnNodeWithPriority) { + return false; + } + + SchedContainerChangeRequest request = + requestsOnNodeWithPriority.remove(containerId); + + // remove hierarchies if it becomes empty + if (requestsOnNodeWithPriority.isEmpty()) { + requestsOnNode.remove(priority); + } + if (requestsOnNode.isEmpty()) { + increaseRequestMap.remove(nodeId); + } + + if (request == null) { + return false; + } + + // update queue's pending resource if request exists + String partition = request.getRMContainer().getNodeLabelExpression(); + Resource delta = request.getDeltaCapacity(); + appResourceUsage.decPending(partition, delta); + queue.decPendingResource(partition, delta); + + if (LOG.isDebugEnabled()) { + LOG.debug("remove increase request:" + request); + } + + return true; + } + + public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId, + Priority priority, ContainerId containerId) { + Map> requestsOnNode = + increaseRequestMap.get(nodeId); + if (null == requestsOnNode) { + return null; + } + + Map requestsOnNodeWithPriority = + requestsOnNode.get(priority); + if (null == requestsOnNodeWithPriority) { + return null; + } + + return requestsOnNodeWithPriority.get(containerId); + } /** * The ApplicationMaster is updating resource requirements for the @@ -163,11 +334,11 @@ synchronized public boolean updateResourceRequests( } } - Map asks = this.requests.get(priority); + Map asks = this.resourceRequestMap.get(priority); if (asks == null) { asks = new ConcurrentHashMap(); - this.requests.put(priority, asks); + this.resourceRequestMap.put(priority, asks); this.priorities.add(priority); } lastRequest = asks.get(resourceName); @@ -260,12 +431,12 @@ void updateUserOrAMBlacklist(Set blacklist, synchronized public Map getResourceRequests( Priority priority) { - return requests.get(priority); + return resourceRequestMap.get(priority); } public List getAllResourceRequests() { List ret = new ArrayList(); - for (Map r : requests.values()) { + for (Map r : resourceRequestMap.values()) { ret.addAll(r.values()); } return ret; @@ -273,7 +444,7 @@ void updateUserOrAMBlacklist(Set blacklist, synchronized public ResourceRequest getResourceRequest(Priority priority, String resourceName) { - Map nodeRequests = requests.get(priority); + Map nodeRequests = resourceRequestMap.get(priority); return (nodeRequests == null) ? null : nodeRequests.get(resourceName); } @@ -301,6 +472,50 @@ public boolean isBlacklisted(String resourceName, } } + public synchronized void increaseContainer( + SchedContainerChangeRequest increaseRequest) { + NodeId nodeId = increaseRequest.getNodeId(); + Priority priority = increaseRequest.getPriority(); + ContainerId containerId = increaseRequest.getContainerId(); + + if (LOG.isDebugEnabled()) { + LOG.debug("allocated increase request : applicationId=" + applicationId + + " container=" + containerId + " host=" + + increaseRequest.getNodeId() + " user=" + user + " resource=" + + increaseRequest.getDeltaCapacity()); + } + + // Set queue metrics + queue.getMetrics().allocateResources(user, 0, + increaseRequest.getDeltaCapacity(), true); + + // remove the increase request from pending increase request map + removeIncreaseRequest(nodeId, priority, containerId); + + // update usage + appResourceUsage.incUsed(increaseRequest.getNodePartition(), + increaseRequest.getDeltaCapacity()); + } + + public synchronized void decreaseContainer( + SchedContainerChangeRequest decreaseRequest) { + // Delta is negative when it's a decrease request + Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity()); + + if (LOG.isDebugEnabled()) { + LOG.debug("Decrease container : applicationId=" + applicationId + + " container=" + decreaseRequest.getContainerId() + " host=" + + decreaseRequest.getNodeId() + " user=" + user + " resource=" + + absDelta); + } + + // Set queue metrics + queue.getMetrics().releaseResources(user, 0, absDelta); + + // update usage + appResourceUsage.decUsed(decreaseRequest.getNodePartition(), absDelta); + } + /** * Resources have been allocated to this application by the resource * scheduler. Track them. @@ -359,11 +574,11 @@ synchronized private void allocateNodeLocal(SchedulerNode node, // Update future requirements decResourceRequest(node.getNodeName(), priority, nodeLocalRequest); - ResourceRequest rackLocalRequest = requests.get(priority).get( + ResourceRequest rackLocalRequest = resourceRequestMap.get(priority).get( node.getRackName()); decResourceRequest(node.getRackName(), priority, rackLocalRequest); - ResourceRequest offRackRequest = requests.get(priority).get( + ResourceRequest offRackRequest = resourceRequestMap.get(priority).get( ResourceRequest.ANY); decrementOutstanding(offRackRequest); @@ -377,7 +592,7 @@ private void decResourceRequest(String resourceName, Priority priority, ResourceRequest request) { request.setNumContainers(request.getNumContainers() - 1); if (request.getNumContainers() == 0) { - requests.get(priority).remove(resourceName); + resourceRequestMap.get(priority).remove(resourceName); } } @@ -394,7 +609,7 @@ synchronized private void allocateRackLocal(SchedulerNode node, // Update future requirements decResourceRequest(node.getRackName(), priority, rackLocalRequest); - ResourceRequest offRackRequest = requests.get(priority).get( + ResourceRequest offRackRequest = resourceRequestMap.get(priority).get( ResourceRequest.ANY); decrementOutstanding(offRackRequest); @@ -449,6 +664,12 @@ synchronized private void checkForDeactivation() { } } } + + // also we need to check increase request + if (!deactivate) { + deactivate = increaseRequestMap.isEmpty(); + } + if (deactivate) { activeUsersManager.deactivateApplication(user, applicationId); } @@ -457,7 +678,7 @@ synchronized private void checkForDeactivation() { synchronized public void move(Queue newQueue) { QueueMetrics oldMetrics = queue.getMetrics(); QueueMetrics newMetrics = newQueue.getMetrics(); - for (Map asks : requests.values()) { + for (Map asks : resourceRequestMap.values()) { ResourceRequest request = asks.get(ResourceRequest.ANY); if (request != null) { oldMetrics.decrPendingResources(user, request.getNumContainers(), @@ -484,7 +705,7 @@ synchronized public void move(Queue newQueue) { synchronized public void stop(RMAppAttemptState rmAppAttemptFinalState) { // clear pending resources metrics for the application QueueMetrics metrics = queue.getMetrics(); - for (Map asks : requests.values()) { + for (Map asks : resourceRequestMap.values()) { ResourceRequest request = asks.get(ResourceRequest.ANY); if (request != null) { metrics.decrPendingResources(user, request.getNumContainers(), 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/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java index 09fd73e..d94b621 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java @@ -373,17 +373,20 @@ public void decrPendingResources(String user, int containers, Resource res) { } private void _decrPendingResources(int containers, Resource res) { + // if #container = 0, means change container resource pendingContainers.decr(containers); - pendingMB.decr(res.getMemory() * containers); - pendingVCores.decr(res.getVirtualCores() * containers); + pendingMB.decr(res.getMemory() * Math.max(containers, 1)); + pendingVCores.decr(res.getVirtualCores() * Math.max(containers, 1)); } public void allocateResources(String user, int containers, Resource res, boolean decrPending) { + // if #containers = 0, means change container resource allocatedContainers.incr(containers); aggregateContainersAllocated.incr(containers); - allocatedMB.incr(res.getMemory() * containers); - allocatedVCores.incr(res.getVirtualCores() * containers); + + allocatedMB.incr(res.getMemory() * Math.max(containers, 1)); + allocatedVCores.incr(res.getVirtualCores() * Math.max(containers, 1)); if (decrPending) { _decrPendingResources(containers, res); } @@ -397,10 +400,11 @@ public void allocateResources(String user, int containers, Resource res, } public void releaseResources(String user, int containers, Resource res) { + // if #container = 0, means change container resource. allocatedContainers.decr(containers); aggregateContainersReleased.incr(containers); - allocatedMB.decr(res.getMemory() * containers); - allocatedVCores.decr(res.getVirtualCores() * containers); + allocatedMB.decr(res.getMemory() * Math.max(containers, 1)); + allocatedVCores.decr(res.getVirtualCores() * Math.max(containers, 1)); QueueMetrics userMetrics = getUserMetrics(user); if (userMetrics != null) { userMetrics.releaseResources(user, containers, res); 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/SchedContainerChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java new file mode 100644 index 0000000..ea109fd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java @@ -0,0 +1,118 @@ +/** + * 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.scheduler; + +import org.apache.hadoop.yarn.api.records.ContainerId; +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.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.util.resource.Resources; + +/** + * This is ContainerResourceChangeRequest in scheduler side, it contains some + * pointers to runtime objects like RMContainer, SchedulerNode, etc. This will + * be easier for scheduler making decision. + */ +public class SchedContainerChangeRequest implements + Comparable { + RMContainer rmContainer; + Resource targetCapacity; + SchedulerNode schedulerNode; + Resource deltaCapacity; + + public SchedContainerChangeRequest(SchedulerNode schedulerNode, + RMContainer rmContainer, Resource targetCapacity) { + this.rmContainer = rmContainer; + this.targetCapacity = targetCapacity; + this.schedulerNode = schedulerNode; + deltaCapacity = Resources.subtract(targetCapacity, + rmContainer.getAllocatedResource()); + } + + public NodeId getNodeId() { + return this.rmContainer.getAllocatedNode(); + } + + public RMContainer getRMContainer() { + return this.rmContainer; + } + + public Resource getTargetCapacity() { + return this.targetCapacity; + } + + /** + * Delta capacity = before - target, so if it is a decrease request, delta + * capacity will be negative + */ + public Resource getDeltaCapacity() { + return deltaCapacity; + } + + public Priority getPriority() { + return rmContainer.getContainer().getPriority(); + } + + public ContainerId getContainerId() { + return rmContainer.getContainerId(); + } + + public String getNodePartition() { + return schedulerNode.getPartition(); + } + + public SchedulerNode getSchedulerNode() { + return schedulerNode; + } + + @Override + public int hashCode() { + return (getContainerId().hashCode() << 16) + targetCapacity.hashCode(); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof SchedContainerChangeRequest)) { + return false; + } + return compareTo((SchedContainerChangeRequest)other) == 0; + } + + @Override + public int compareTo(SchedContainerChangeRequest other) { + if (other == null) { + return -1; + } + + int rc = getPriority().compareTo(other.getPriority()); + if (0 != rc) { + return rc; + } + + return getContainerId().compareTo(other.getContainerId()); + } + + @Override + public String toString() { + return ""; + } +} 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/SchedulerApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java index 519de98..96288f8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java @@ -28,7 +28,7 @@ private Queue queue; private final String user; - private T currentAttempt; + private volatile T currentAttempt; private volatile Priority priority; public SchedulerApplication(Queue queue, String user) { 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 b361d15..f064e97 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 @@ -19,11 +19,13 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -51,16 +53,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; 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.RMContainerChangeResourceEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity; +import org.apache.hadoop.yarn.state.InvalidStateTransitionException; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -104,8 +109,10 @@ private AtomicLong firstAllocationRequestSentTime = new AtomicLong(0); private AtomicLong firstContainerAllocatedTime = new AtomicLong(0); - protected List newlyAllocatedContainers = - new ArrayList(); + protected List newlyAllocatedContainers = new ArrayList<>(); + protected Map newlyDecreasedContainers = new HashMap<>(); + protected Map newlyIncreasedContainers = new HashMap<>(); + protected Set updatedNMTokens = new HashSet<>(); // This pendingRelease is used in work-preserving recovery scenario to keep // track of the AM's outstanding release requests. RM on recovery could @@ -219,7 +226,8 @@ public long getNewContainerId() { return appSchedulingInfo.getPriorities(); } - public synchronized ResourceRequest getResourceRequest(Priority priority, String resourceName) { + public synchronized ResourceRequest getResourceRequest(Priority priority, + String resourceName) { return this.appSchedulingInfo.getResourceRequest(priority, resourceName); } @@ -324,24 +332,28 @@ public synchronized boolean isStopped() { return reservedContainers; } - public synchronized RMContainer reserve(SchedulerNode node, Priority priority, - RMContainer rmContainer, Container container) { - // Create RMContainer if necessary - if (rmContainer == null) { - rmContainer = - new RMContainerImpl(container, getApplicationAttemptId(), - node.getNodeID(), appSchedulingInfo.getUser(), rmContext); + public synchronized boolean reserveIncreasedContainer(SchedulerNode node, + Priority priority, RMContainer rmContainer, Resource reservedResource) { + if (commonReserve(node, priority, rmContainer, reservedResource)) { attemptResourceUsage.incReserved(node.getPartition(), - container.getResource()); - - // Reset the re-reservation count - resetReReservations(priority); - } else { - // Note down the re-reservation - addReReservation(priority); + reservedResource); + // succeeded + return true; + } + + return false; + } + + private synchronized boolean commonReserve(SchedulerNode node, + Priority priority, RMContainer rmContainer, Resource reservedResource) { + try { + rmContainer.handle(new RMContainerReservedEvent(rmContainer + .getContainerId(), reservedResource, node.getNodeID(), priority)); + } catch (InvalidStateTransitionException e) { + // We reach here could be caused by container already finished, return + // false indicate it fails + return false; } - rmContainer.handle(new RMContainerReservedEvent(container.getId(), - container.getResource(), node.getNodeID(), priority)); Map reservedContainers = this.reservedContainers.get(priority); @@ -356,8 +368,30 @@ public synchronized RMContainer reserve(SchedulerNode node, Priority priority, + " reserved container " + rmContainer + " on node " + node + ". This attempt currently has " + reservedContainers.size() + " reserved containers at priority " + priority - + "; currentReservation " + container.getResource()); + + "; currentReservation " + reservedResource); } + + return true; + } + + public synchronized RMContainer reserve(SchedulerNode node, + Priority priority, RMContainer rmContainer, Container container) { + // Create RMContainer if necessary + if (rmContainer == null) { + rmContainer = + new RMContainerImpl(container, getApplicationAttemptId(), + node.getNodeID(), appSchedulingInfo.getUser(), rmContext); + attemptResourceUsage.incReserved(node.getPartition(), + container.getResource()); + + // Reset the re-reservation count + resetReReservations(priority); + } else { + // Note down the re-reservation + addReReservation(priority); + } + + commonReserve(node, priority, rmContainer, container.getResource()); return rmContainer; } @@ -437,69 +471,100 @@ public synchronized void showRequests() { public Resource getCurrentConsumption() { return attemptResourceUsage.getUsed(); } - - public static class ContainersAndNMTokensAllocation { - List containerList; - List nmTokenList; - - public ContainersAndNMTokensAllocation(List containerList, - List nmTokenList) { - this.containerList = containerList; - this.nmTokenList = nmTokenList; + + private Container updateContainerAndNMToken(RMContainer rmContainer, + boolean newContainer, boolean increasedContainer) { + Container container = rmContainer.getContainer(); + ContainerType containerType = ContainerType.TASK; + // The working knowledge is that masterContainer for AM is null as it + // itself is the master container. + RMAppAttempt appAttempt = rmContext.getRMApps() + .get(container.getId().getApplicationAttemptId().getApplicationId()) + .getCurrentAppAttempt(); + if (isWaitingForAMContainer(getApplicationId())) { + containerType = ContainerType.APPLICATION_MASTER; } - - public List getContainerList() { - return containerList; + try { + // create container token and NMToken altogether. + container.setContainerToken(rmContext.getContainerTokenSecretManager() + .createContainerToken(container.getId(), container.getNodeId(), + getUser(), container.getResource(), container.getPriority(), + rmContainer.getCreationTime(), this.logAggregationContext, + rmContainer.getNodeLabelExpression(), containerType)); + NMToken nmToken = + rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(), + getApplicationAttemptId(), container); + if (nmToken != null) { + updatedNMTokens.add(nmToken); + } + } catch (IllegalArgumentException e) { + // DNS might be down, skip returning this container. + LOG.error("Error trying to assign container token and NM token to" + + " an updated container " + container.getId(), e); + return null; } - - public List getNMTokenList() { - return nmTokenList; + + if (newContainer) { + rmContainer.handle(new RMContainerEvent( + rmContainer.getContainerId(), RMContainerEventType.ACQUIRED)); + } else { + rmContainer.handle(new RMContainerUpdatesAcquiredEvent( + rmContainer.getContainerId(), increasedContainer)); } + return container; } - // Create container token and NMToken altogether, if either of them fails for + // Create container token and update NMToken altogether, if either of them fails for // some reason like DNS unavailable, do not return this container and keep it // in the newlyAllocatedContainers waiting to be refetched. - public synchronized ContainersAndNMTokensAllocation - pullNewlyAllocatedContainersAndNMTokens() { + public synchronized List pullNewlyAllocatedContainers() { List returnContainerList = new ArrayList(newlyAllocatedContainers.size()); - List nmTokens = new ArrayList(); for (Iterator i = newlyAllocatedContainers.iterator(); i - .hasNext();) { + .hasNext();) { RMContainer rmContainer = i.next(); - Container container = rmContainer.getContainer(); - ContainerType containerType = ContainerType.TASK; - boolean isWaitingForAMContainer = isWaitingForAMContainer( - container.getId().getApplicationAttemptId().getApplicationId()); - if (isWaitingForAMContainer) { - containerType = ContainerType.APPLICATION_MASTER; + Container updatedContainer = + updateContainerAndNMToken(rmContainer, true, false); + // Only add container to return list when it's not null. updatedContainer + // could be null when generate token failed, it can be caused by DNS + // resolving failed. + if (updatedContainer != null) { + returnContainerList.add(updatedContainer); + i.remove(); } - try { - // create container token and NMToken altogether. - container.setContainerToken(rmContext.getContainerTokenSecretManager() - .createContainerToken(container.getId(), container.getNodeId(), - getUser(), container.getResource(), container.getPriority(), - rmContainer.getCreationTime(), this.logAggregationContext, - rmContainer.getNodeLabelExpression(), containerType)); - NMToken nmToken = - rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(), - getApplicationAttemptId(), container); - if (nmToken != null) { - nmTokens.add(nmToken); - } - } catch (IllegalArgumentException e) { - // DNS might be down, skip returning this container. - LOG.error("Error trying to assign container token and NM token to" + - " an allocated container " + container.getId(), e); - continue; + } + return returnContainerList; + } + + private synchronized List pullNewlyUpdatedContainers( + Map updatedContainerMap, boolean increase) { + List returnContainerList = + new ArrayList(updatedContainerMap.size()); + for (Iterator> i = + updatedContainerMap.entrySet().iterator(); i.hasNext();) { + RMContainer rmContainer = i.next().getValue(); + Container updatedContainer = + updateContainerAndNMToken(rmContainer, false, increase); + if (updatedContainer != null) { + returnContainerList.add(updatedContainer); + i.remove(); } - returnContainerList.add(container); - i.remove(); - rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(), - RMContainerEventType.ACQUIRED)); } - return new ContainersAndNMTokensAllocation(returnContainerList, nmTokens); + return returnContainerList; + } + + public synchronized List pullNewlyIncreasedContainers() { + return pullNewlyUpdatedContainers(newlyIncreasedContainers, true); + } + + public synchronized List pullNewlyDecreasedContainers() { + return pullNewlyUpdatedContainers(newlyDecreasedContainers, false); + } + + public synchronized List pullUpdatedNMTokens() { + List returnList = new ArrayList(updatedNMTokens); + updatedNMTokens.clear(); + return returnList; } public boolean isWaitingForAMContainer(ApplicationId applicationId) { @@ -770,4 +835,50 @@ public synchronized ResourceUsage getSchedulingResourceUsage() { return attemptResourceUsage; } -} + public synchronized boolean removeIncreaseRequest(NodeId nodeId, + Priority priority, ContainerId containerId) { + return appSchedulingInfo.removeIncreaseRequest(nodeId, priority, + containerId); + } + + public synchronized boolean updateIncreaseRequests( + List increaseRequests) { + return appSchedulingInfo.updateIncreaseRequests(increaseRequests); + } + + private synchronized void changeContainerResource( + SchedContainerChangeRequest changeRequest, boolean increase) { + if (increase) { + appSchedulingInfo.increaseContainer(changeRequest); + } else { + appSchedulingInfo.decreaseContainer(changeRequest); + } + + RMContainer changedRMContainer = changeRequest.getRMContainer(); + changedRMContainer.handle( + new RMContainerChangeResourceEvent(changeRequest.getContainerId(), + changeRequest.getTargetCapacity(), increase)); + + // remove pending and not pulled by AM newly-increased/decreased-containers + // and add the new one + if (increase) { + newlyDecreasedContainers.remove(changeRequest.getContainerId()); + newlyIncreasedContainers.put(changeRequest.getContainerId(), + changedRMContainer); + } else { + newlyIncreasedContainers.remove(changeRequest.getContainerId()); + newlyDecreasedContainers.put(changeRequest.getContainerId(), + changedRMContainer); + } + } + + public synchronized void decreaseContainer( + SchedContainerChangeRequest decreaseRequest) { + changeContainerResource(decreaseRequest, false); + } + + public synchronized void increaseContainer( + SchedContainerChangeRequest increaseRequest) { + changeContainerResource(increaseRequest, true); + } +} \ No newline at end of file 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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java index f03663a..f3d3906 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java @@ -157,6 +157,37 @@ public synchronized void allocateContainer(RMContainer rmContainer) { + getUsedResource() + " used and " + getAvailableResource() + " available after allocation"); } + + private synchronized void changeContainerResource(ContainerId containerId, + Resource deltaResource, boolean increase) { + if (increase) { + deductAvailableResource(deltaResource); + } else { + addAvailableResource(deltaResource); + } + + LOG.info((increase ? "Increased" : "Decreased") + " container " + + containerId + " of capacity " + deltaResource + " on host " + + rmNode.getNodeAddress() + ", which has " + numContainers + + " containers, " + getUsedResource() + " used and " + + getAvailableResource() + " available after allocation"); + } + + /** + * The Scheduler increased container + */ + public synchronized void increaseContainer(ContainerId containerId, + Resource deltaResource) { + changeContainerResource(containerId, deltaResource, true); + } + + /** + * The Scheduler decreased container + */ + public synchronized void decreaseContainer(ContainerId containerId, + Resource deltaResource) { + changeContainerResource(containerId, deltaResource, false); + } /** * Get available resources on the node. 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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java index 8047d0b..abefee8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java @@ -361,7 +361,7 @@ public static AccessType toAccessType(QueueACL acl) { } public static boolean checkResourceRequestMatchingNodePartition( - ResourceRequest offswitchResourceRequest, String nodePartition, + String requestedPartition, String nodePartition, SchedulingMode schedulingMode) { // We will only look at node label = nodeLabelToLookAt according to // schedulingMode and partition of node. @@ -371,12 +371,11 @@ public static boolean checkResourceRequestMatchingNodePartition( } else { nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL; } - - String askedNodePartition = offswitchResourceRequest.getNodeLabelExpression(); - if (null == askedNodePartition) { - askedNodePartition = RMNodeLabelsManager.NO_LABEL; + + if (null == requestedPartition) { + requestedPartition = RMNodeLabelsManager.NO_LABEL; } - return askedNodePartition.equals(nodePartitionToLookAt); + return requestedPartition.equals(nodePartitionToLookAt); } private static boolean hasPendingResourceRequest(ResourceCalculator rc, 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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java index f2753e6..099db69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java @@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.QueueACL; @@ -133,16 +134,17 @@ public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues, * @param release * @param blacklistAdditions * @param blacklistRemovals + * @param increaseRequests + * @param decreaseRequests * @return the {@link Allocation} for the application */ @Public @Stable - Allocation - allocate(ApplicationAttemptId appAttemptId, - List ask, - List release, - List blacklistAdditions, - List blacklistRemovals); + Allocation allocate(ApplicationAttemptId appAttemptId, + List ask, List release, + List blacklistAdditions, List blacklistRemovals, + List increaseRequests, + List decreaseRequests); /** * Get node resource usage report. 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/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 0ae4d1a..9f61b11 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -43,10 +43,10 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -76,7 +76,7 @@ private boolean preemptionDisabled; // Track resource usage-by-label like used-resource/pending-resource, etc. - ResourceUsage queueUsage; + volatile ResourceUsage queueUsage; // Track capacities like used-capcity/abs-used-capacity/capacity/abs-capacity, // etc. @@ -340,22 +340,27 @@ public Resource getMinimumAllocation() { return minimumAllocation; } - synchronized void allocateResource(Resource clusterResource, - Resource resource, String nodePartition) { + synchronized void allocateResource(Resource clusterResource, + Resource resource, String nodePartition, boolean changeContainerResource) { queueUsage.incUsed(nodePartition, resource); - ++numContainers; + if (!changeContainerResource) { + ++numContainers; + } CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, minimumAllocation, this, labelManager, nodePartition); } protected synchronized void releaseResource(Resource clusterResource, - Resource resource, String nodePartition) { + Resource resource, String nodePartition, boolean changeContainerResource) { queueUsage.decUsed(nodePartition, resource); CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource, minimumAllocation, this, labelManager, nodePartition); - --numContainers; + + if (!changeContainerResource) { + --numContainers; + } } @Private @@ -446,8 +451,8 @@ private Resource getCurrentLimitResource(String nodePartition, } synchronized boolean canAssignToThisQueue(Resource clusterResource, - String nodePartition, ResourceLimits currentResourceLimits, Resource resourceCouldBeUnreserved, - SchedulingMode schedulingMode) { + String nodePartition, ResourceLimits currentResourceLimits, + Resource resourceCouldBeUnreserved, SchedulingMode schedulingMode) { // Get current limited resource: // - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect // queues' max capacity. 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/capacity/CSAssignment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java index 928437f..68f6f12 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java @@ -41,6 +41,7 @@ private final boolean skipped; private boolean fulfilledReservation; private final AssignmentInformation assignmentInformation; + private boolean increaseAllocation; public CSAssignment(Resource resource, NodeType type) { this(resource, type, null, null, false, false); @@ -138,4 +139,12 @@ public boolean isFulfilledReservation() { public AssignmentInformation getAssignmentInformation() { return this.assignmentInformation; } + + public boolean isIncreasedAllocation() { + return increaseAllocation; + } + + public void setIncreasedAllocation(boolean flag) { + increaseAllocation = flag; + } } \ No newline at end of file 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/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 9855dd4..e90deeb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; @@ -219,6 +220,14 @@ public void completedContainer(Resource clusterResource, boolean sortQueues); /** + * We have a reserved increased container in the queue, we need to unreserve + * it. Since we just want to cancel the reserved increase request instead of + * stop the container, we shouldn't call completedContainer for such purpose. + */ + public void unreserveIncreasedContainer(Resource clusterResource, + FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer); + + /** * Get the number of applications in the queue. * @return number of applications */ @@ -313,4 +322,11 @@ public void decUsedResource(String nodePartition, Resource resourceToDec, * new resource asked */ public void decPendingResource(String nodeLabel, Resource resourceToDec); + + /** + * Decrease container resource in the queue + */ + public void decreaseContainer(Resource clusterResource, + SchedContainerChangeRequest decreaseRequest, + FiCaSchedulerApp app); } 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/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index ad5c76c..465e233 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.EnumSet; @@ -52,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -87,6 +89,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; @@ -98,6 +101,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueNotFoundException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; @@ -840,10 +845,14 @@ private synchronized void doneApplicationAttempt( } @Override + // Note: when AM asks to decrease container or release container, we will + // acquire scheduler lock @Lock(Lock.NoLock.class) public Allocation allocate(ApplicationAttemptId applicationAttemptId, - List ask, List release, - List blacklistAdditions, List blacklistRemovals) { + List ask, List release, + List blacklistAdditions, List blacklistRemovals, + List increaseRequests, + List decreaseRequests) { FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId); if (application == null) { @@ -854,6 +863,14 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, SchedulerUtils.normalizeRequests( ask, getResourceCalculator(), getClusterResource(), getMinimumResourceCapability(), getMaximumResourceCapability()); + + // Pre-process increase requests + List normalizedIncreaseRequests = + checkAndNormalizeContainerChangeRequests(increaseRequests, true); + + // Pre-process decrease requests + List normalizedDecreaseRequests = + checkAndNormalizeContainerChangeRequests(decreaseRequests, false); // Release containers releaseContainers(release, application); @@ -870,8 +887,8 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, return EMPTY_ALLOCATION; } + // Process resource requests if (!ask.isEmpty()) { - if(LOG.isDebugEnabled()) { LOG.debug("allocate: pre-update " + applicationAttemptId + " ask size =" + ask.size()); @@ -888,6 +905,12 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, application.showRequests(); } } + + // Process increase resource requests + if (application.updateIncreaseRequests(normalizedIncreaseRequests) + && (updateDemandForQueue == null)) { + updateDemandForQueue = (LeafQueue) application.getQueue(); + } if (application.isWaitingForAMContainer(application.getApplicationId())) { // Allocate is for AM and update AM blacklist for this @@ -896,6 +919,9 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, } else { application.updateBlacklist(blacklistAdditions, blacklistRemovals); } + + // Decrease containers + decreaseContainers(normalizedDecreaseRequests, application); allocation = application.getAllocation(getResourceCalculator(), clusterResource, getMinimumResourceCapability()); @@ -957,6 +983,13 @@ private synchronized void nodeUpdate(RMNode nm) { for (ContainerStatus launchedContainer : newlyLaunchedContainers) { containerLaunchedOnNode(launchedContainer.getContainerId(), node); } + + // Processing the newly increased containers + List newlyIncreasedContainers = + nm.pullNewlyIncreasedContainers(); + for (Container container : newlyIncreasedContainers) { + containerIncreasedOnNode(container.getId(), node, container); + } // Process completed containers int releasedContainers = 0; @@ -1442,6 +1475,50 @@ protected synchronized void completedContainer(RMContainer rmContainer, container.getId(), queue.getQueuePath()); } } + + @Lock(CapacityScheduler.class) + @Override + protected synchronized void decreaseContainer( + SchedContainerChangeRequest decreaseRequest, + SchedulerApplicationAttempt attempt) { + RMContainer rmContainer = decreaseRequest.getRMContainer(); + + // Check container status before doing decrease + if (rmContainer.getState() != RMContainerState.RUNNING) { + LOG.info("Trying to decrease a container not in RUNNING state, container=" + + rmContainer + " state=" + rmContainer.getState().name()); + return; + } + + // Delta capacity of this decrease request is 0, this decrease request may + // just to cancel increase request + if (Resources.equals(decreaseRequest.getDeltaCapacity(), Resources.none())) { + if (LOG.isDebugEnabled()) { + LOG.debug("Decrease target resource equals to existing resource for container:" + + decreaseRequest.getContainerId() + + " ignore this decrease request."); + } + return; + } + + // Save resource before decrease + Resource resourceBeforeDecrease = + Resources.clone(rmContainer.getContainer().getResource()); + + FiCaSchedulerApp app = (FiCaSchedulerApp)attempt; + LeafQueue queue = (LeafQueue) attempt.getQueue(); + queue.decreaseContainer(clusterResource, decreaseRequest, app); + + // Notify RMNode the container will be decreased + this.rmContext.getDispatcher().getEventHandler() + .handle(new RMNodeDecreaseContainerEvent(decreaseRequest.getNodeId(), + Arrays.asList(rmContainer.getContainer()))); + + LOG.info("Application attempt " + app.getApplicationAttemptId() + + " decreased container:" + decreaseRequest.getContainerId() + " from " + + resourceBeforeDecrease + " to " + + decreaseRequest.getTargetCapacity()); + } @Lock(Lock.NoLock.class) @VisibleForTesting 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/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java index b43f658..15d7c32 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -60,10 +59,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.Lock; @@ -730,17 +729,22 @@ private synchronized FiCaSchedulerApp getApplication( } private void handleExcessReservedContainer(Resource clusterResource, - CSAssignment assignment) { + CSAssignment assignment, FiCaSchedulerNode node, FiCaSchedulerApp app) { if (assignment.getExcessReservation() != null) { RMContainer excessReservedContainer = assignment.getExcessReservation(); - - completedContainer(clusterResource, assignment.getApplication(), - scheduler.getNode(excessReservedContainer.getAllocatedNode()), - excessReservedContainer, - SchedulerUtils.createAbnormalContainerStatus( - excessReservedContainer.getContainerId(), - SchedulerUtils.UNRESERVED_CONTAINER), - RMContainerEventType.RELEASED, null, false); + + if (excessReservedContainer.hasIncreaseReservation()) { + unreserveIncreasedContainer(clusterResource, + app, node, excessReservedContainer); + } else { + completedContainer(clusterResource, assignment.getApplication(), + scheduler.getNode(excessReservedContainer.getAllocatedNode()), + excessReservedContainer, + SchedulerUtils.createAbnormalContainerStatus( + excessReservedContainer.getContainerId(), + SchedulerUtils.UNRESERVED_CONTAINER), + RMContainerEventType.RELEASED, null, false); + } assignment.setExcessReservation(null); } @@ -766,7 +770,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, CSAssignment assignment = application.assignContainers(clusterResource, node, currentResourceLimits, schedulingMode, reservedContainer); - handleExcessReservedContainer(clusterResource, assignment); + handleExcessReservedContainer(clusterResource, assignment, node, + application); return assignment; } } @@ -824,7 +829,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, // Did we schedule or reserve a container? Resource assigned = assignment.getResource(); - handleExcessReservedContainer(clusterResource, assignment); + handleExcessReservedContainer(clusterResource, assignment, node, + application); if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, Resources.none())) { @@ -836,7 +842,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, // Book-keeping // Note: Update headroom to account for current allocation too... allocateResource(clusterResource, application, assigned, - node.getPartition(), reservedOrAllocatedRMContainer); + node.getPartition(), reservedOrAllocatedRMContainer, + assignment.isIncreasedAllocation()); // Done return assignment; @@ -1086,6 +1093,37 @@ protected synchronized boolean canAssignToUser(Resource clusterResource, } return true; } + + @Override + public void unreserveIncreasedContainer(Resource clusterResource, + FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) { + boolean removed = false; + Priority priority = null; + + synchronized (this) { + if (rmContainer.getContainer() != null) { + priority = rmContainer.getContainer().getPriority(); + } + + if (null != priority) { + removed = app.unreserve(rmContainer.getContainer().getPriority(), node, + rmContainer); + } + + if (removed) { + // Inform the ordering policy + orderingPolicy.containerReleased(app, rmContainer); + + releaseResource(clusterResource, app, rmContainer.getReservedResource(), + node.getPartition(), rmContainer, true); + } + } + + if (removed) { + getParent().unreserveIncreasedContainer(clusterResource, app, node, + rmContainer); + } + } @Override public void completedContainer(Resource clusterResource, @@ -1093,6 +1131,15 @@ public void completedContainer(Resource clusterResource, ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue, boolean sortQueues) { if (application != null) { + // unreserve container increase request if it previously reserved. + if (rmContainer.hasIncreaseReservation()) { + unreserveIncreasedContainer(clusterResource, application, node, + rmContainer); + } + + // Remove container increase request if it exists + application.removeIncreaseRequest(node.getNodeID(), + rmContainer.getAllocatedPriority(), rmContainer.getContainerId()); boolean removed = false; @@ -1123,7 +1170,7 @@ public void completedContainer(Resource clusterResource, orderingPolicy.containerReleased(application, rmContainer); releaseResource(clusterResource, application, container.getResource(), - node.getPartition(), rmContainer); + node.getPartition(), rmContainer, false); } } @@ -1137,8 +1184,10 @@ public void completedContainer(Resource clusterResource, synchronized void allocateResource(Resource clusterResource, SchedulerApplicationAttempt application, Resource resource, - String nodePartition, RMContainer rmContainer) { - super.allocateResource(clusterResource, resource, nodePartition); + String nodePartition, RMContainer rmContainer, + boolean isIncreasedAllocation) { + super.allocateResource(clusterResource, resource, nodePartition, + isIncreasedAllocation); // handle ignore exclusivity container if (null != rmContainer && rmContainer.getNodeLabelExpression().equals( @@ -1174,8 +1223,9 @@ synchronized void allocateResource(Resource clusterResource, synchronized void releaseResource(Resource clusterResource, FiCaSchedulerApp application, Resource resource, String nodePartition, - RMContainer rmContainer) { - super.releaseResource(clusterResource, resource, nodePartition); + RMContainer rmContainer, boolean isChangeResource) { + super.releaseResource(clusterResource, resource, nodePartition, + isChangeResource); // handle ignore exclusivity container if (null != rmContainer && rmContainer.getNodeLabelExpression().equals( @@ -1363,7 +1413,7 @@ public void recoverContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, attempt, rmContainer.getContainer() - .getResource(), node.getPartition(), rmContainer); + .getResource(), node.getPartition(), rmContainer, false); } getParent().recoverContainer(clusterResource, attempt, rmContainer); } @@ -1412,7 +1462,7 @@ public void attachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); allocateResource(clusterResource, application, rmContainer.getContainer() - .getResource(), node.getPartition(), rmContainer); + .getResource(), node.getPartition(), rmContainer, false); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() + " resource=" + rmContainer.getContainer().getResource() + " queueMoveIn=" + this + " usedCapacity=" + getUsedCapacity() @@ -1430,7 +1480,7 @@ public void detachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); releaseResource(clusterResource, application, rmContainer.getContainer() - .getResource(), node.getPartition(), rmContainer); + .getResource(), node.getPartition(), rmContainer, false); LOG.info("movedContainer" + " container=" + rmContainer.getContainer() + " resource=" + rmContainer.getContainer().getResource() + " queueMoveOut=" + this + " usedCapacity=" + getUsedCapacity() @@ -1482,6 +1532,39 @@ public synchronized void setOrderingPolicy( public Priority getDefaultApplicationPriority() { return defaultAppPriorityPerQueue; } + + @Override + public void decreaseContainer(Resource clusterResource, + SchedContainerChangeRequest decreaseRequest, + FiCaSchedulerApp app) { + // If the container being decreased is reserved, we need to unreserve it + // first. + RMContainer rmContainer = decreaseRequest.getRMContainer(); + if (rmContainer.hasIncreaseReservation()) { + unreserveIncreasedContainer(clusterResource, app, + (FiCaSchedulerNode)decreaseRequest.getSchedulerNode(), rmContainer); + } + + // Delta capacity is negative when it's a decrease request + Resource absDelta = Resources.negate(decreaseRequest.getDeltaCapacity()); + + synchronized (this) { + // Delta is negative when it's a decrease request + releaseResource(clusterResource, app, absDelta, + decreaseRequest.getNodePartition(), decreaseRequest.getRMContainer(), + true); + // Notify application + app.decreaseContainer(decreaseRequest); + // Notify node + decreaseRequest.getSchedulerNode() + .decreaseContainer(decreaseRequest.getContainerId(), absDelta); + } + + // Notify parent + if (getParent() != null) { + getParent().decreaseContainer(clusterResource, decreaseRequest, app); + } + } public synchronized OrderingPolicy getPendingAppsOrderingPolicy() { 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/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index e01204cc..badab72 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.resource.Resources; @@ -430,7 +431,7 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, assignedToChild.getResource(), Resources.none())) { // Track resource utilization for the parent-queue super.allocateResource(clusterResource, assignedToChild.getResource(), - node.getPartition()); + node.getPartition(), assignedToChild.isIncreasedAllocation()); // Track resource utilization in this pass of the scheduler Resources @@ -454,6 +455,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, .addAll( assignedToChild.getAssignmentInformation() .getReservationDetails()); + assignment.setIncreasedAllocation(assignedToChild + .isIncreasedAllocation()); LOG.info("assignedContainer" + " queue=" + getQueueName() + @@ -616,6 +619,73 @@ private void printChildQueues() { } } + private synchronized void internalReleaseResource(Resource clusterResource, + FiCaSchedulerNode node, Resource releasedResource, boolean changeResource, + CSQueue completedChildQueue, boolean sortQueues) { + super.releaseResource(clusterResource, + releasedResource, node.getPartition(), + changeResource); + + if (LOG.isDebugEnabled()) { + LOG.debug("completedContainer " + this + ", cluster=" + clusterResource); + } + + // Note that this is using an iterator on the childQueues so this can't + // be called if already within an iterator for the childQueues. Like + // from assignContainersToChildQueues. + if (sortQueues) { + // reinsert the updated queue + for (Iterator iter = childQueues.iterator(); iter.hasNext();) { + CSQueue csqueue = iter.next(); + if (csqueue.equals(completedChildQueue)) { + iter.remove(); + if (LOG.isDebugEnabled()) { + LOG.debug("Re-sorting completed queue: " + csqueue); + } + childQueues.add(csqueue); + break; + } + } + } + + // If we skipped sort queue this time, we need to resort queues to make + // sure we allocate from least usage (or order defined by queue policy) + // queues. + needToResortQueuesAtNextAllocation = !sortQueues; + } + + @Override + public void decreaseContainer(Resource clusterResource, + SchedContainerChangeRequest decreaseRequest, FiCaSchedulerApp app) { + // delta capacity is negative when it's a decrease request + Resource absDeltaCapacity = + Resources.negate(decreaseRequest.getDeltaCapacity()); + + internalReleaseResource(clusterResource, + csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false, + null, false); + + // Inform the parent + if (parent != null) { + parent.decreaseContainer(clusterResource, decreaseRequest, app); + } + } + + @Override + public void unreserveIncreasedContainer(Resource clusterResource, + FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) { + if (app != null) { + internalReleaseResource(clusterResource, node, + rmContainer.getReservedResource(), false, null, false); + + // Inform the parent + if (parent != null) { + parent.unreserveIncreasedContainer(clusterResource, app, node, + rmContainer); + } + } + } + @Override public void completedContainer(Resource clusterResource, FiCaSchedulerApp application, FiCaSchedulerNode node, @@ -623,40 +693,9 @@ public void completedContainer(Resource clusterResource, RMContainerEventType event, CSQueue completedChildQueue, boolean sortQueues) { if (application != null) { - // Careful! Locking order is important! - // Book keeping - synchronized (this) { - super.releaseResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getPartition()); - - if (LOG.isDebugEnabled()) { - LOG.debug("completedContainer " + this + ", cluster=" + clusterResource); - } - - // Note that this is using an iterator on the childQueues so this can't - // be called if already within an iterator for the childQueues. Like - // from assignContainersToChildQueues. - if (sortQueues) { - // reinsert the updated queue - for (Iterator iter = childQueues.iterator(); - iter.hasNext();) { - CSQueue csqueue = iter.next(); - if(csqueue.equals(completedChildQueue)) { - iter.remove(); - if (LOG.isDebugEnabled()) { - LOG.debug("Re-sorting completed queue: " + csqueue); - } - childQueues.add(csqueue); - break; - } - } - } - - // If we skipped sort queue this time, we need to resort queues to make - // sure we allocate from least usage (or order defined by queue policy) - // queues. - needToResortQueuesAtNextAllocation = !sortQueues; - } + internalReleaseResource(clusterResource, node, + rmContainer.getContainer().getResource(), false, completedChildQueue, + sortQueues); // Inform the parent if (parent != null) { @@ -698,7 +737,7 @@ public void recoverContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); super.allocateResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getPartition()); + .getResource(), node.getPartition(), false); } if (parent != null) { parent.recoverContainer(clusterResource, attempt, rmContainer); @@ -726,7 +765,7 @@ public void attachContainer(Resource clusterResource, FiCaSchedulerNode node = scheduler.getNode(rmContainer.getContainer().getNodeId()); super.allocateResource(clusterResource, rmContainer.getContainer() - .getResource(), node.getPartition()); + .getResource(), node.getPartition(), false); LOG.info("movedContainer" + " queueMoveIn=" + getQueueName() + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" @@ -746,7 +785,7 @@ public void detachContainer(Resource clusterResource, scheduler.getNode(rmContainer.getContainer().getNodeId()); super.releaseResource(clusterResource, rmContainer.getContainer().getResource(), - node.getPartition()); + node.getPartition(), false); LOG.info("movedContainer" + " queueMoveOut=" + getQueueName() + " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed() + " cluster=" 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/capacity/allocator/AbstractContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java new file mode 100644 index 0000000..b986b1f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java @@ -0,0 +1,131 @@ +/** + * 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.scheduler.capacity.allocator; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; + +/** + * For an application, resource limits and resource requests, decide how to + * allocate container. This is to make application resource allocation logic + * extensible. + */ +public abstract class AbstractContainerAllocator { + private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class); + + FiCaSchedulerApp application; + final ResourceCalculator rc; + final RMContext rmContext; + + public AbstractContainerAllocator(FiCaSchedulerApp application, + ResourceCalculator rc, RMContext rmContext) { + this.application = application; + this.rc = rc; + this.rmContext = rmContext; + } + + protected CSAssignment getCSAssignmentFromAllocateResult( + Resource clusterResource, ContainerAllocation result, + RMContainer rmContainer) { + // Handle skipped + boolean skipped = + (result.getAllocationState() == AllocationState.APP_SKIPPED); + CSAssignment assignment = new CSAssignment(skipped); + assignment.setApplication(application); + + // Handle excess reservation + assignment.setExcessReservation(result.getContainerToBeUnreserved()); + + // If we allocated something + if (Resources.greaterThan(rc, clusterResource, + result.getResourceToBeAllocated(), Resources.none())) { + Resource allocatedResource = result.getResourceToBeAllocated(); + Container updatedContainer = result.getUpdatedContainer(); + + assignment.setResource(allocatedResource); + assignment.setType(result.getContainerNodeType()); + + if (result.getAllocationState() == AllocationState.RESERVED) { + // This is a reserved container + LOG.info("Reserved container " + " application=" + + application.getApplicationId() + " resource=" + allocatedResource + + " queue=" + this.toString() + " cluster=" + clusterResource); + assignment.getAssignmentInformation().addReservationDetails( + updatedContainer.getId(), + application.getCSLeafQueue().getQueuePath()); + assignment.getAssignmentInformation().incrReservations(); + Resources.addTo(assignment.getAssignmentInformation().getReserved(), + allocatedResource); + } else if (result.getAllocationState() == AllocationState.ALLOCATED){ + // This is a new container + // Inform the ordering policy + LOG.info("assignedContainer" + " application attempt=" + + application.getApplicationAttemptId() + " container=" + + updatedContainer.getId() + " queue=" + this + " clusterResource=" + + clusterResource); + + application + .getCSLeafQueue() + .getOrderingPolicy() + .containerAllocated(application, + application.getRMContainer(updatedContainer.getId())); + + assignment.getAssignmentInformation().addAllocationDetails( + updatedContainer.getId(), + application.getCSLeafQueue().getQueuePath()); + assignment.getAssignmentInformation().incrAllocations(); + Resources.addTo(assignment.getAssignmentInformation().getAllocated(), + allocatedResource); + + if (rmContainer != null) { + assignment.setFulfilledReservation(true); + } + } + } + + return assignment; + } + + /** + * allocate needs to handle following stuffs: + * + *
      + *
    • Select request: Select a request to allocate. E.g. select a resource + * request based on requirement/priority/locality.
    • + *
    • Check if a given resource can be allocated based on resource + * availability
    • + *
    • Do allocation: this will decide/create allocated/reserved + * container, this will also update metrics
    • + *
    + */ + public abstract CSAssignment assignContainers(Resource clusterResource, + FiCaSchedulerNode node, SchedulingMode schedulingMode, + ResourceLimits resourceLimits, RMContainer reservedContainer); +} \ No newline at end of file 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/capacity/allocator/ContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java index 6e296cd..3be8e0e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocator.java @@ -18,13 +18,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; -import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; @@ -33,118 +30,50 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -/** - * For an application, resource limits and resource requests, decide how to - * allocate container. This is to make application resource allocation logic - * extensible. - */ -public abstract class ContainerAllocator { - private static final Log LOG = LogFactory.getLog(ContainerAllocator.class); +public class ContainerAllocator extends AbstractContainerAllocator { + AbstractContainerAllocator increaseContainerAllocator; + AbstractContainerAllocator regularContainerAllocator; - FiCaSchedulerApp application; - final ResourceCalculator rc; - final RMContext rmContext; - public ContainerAllocator(FiCaSchedulerApp application, ResourceCalculator rc, RMContext rmContext) { - this.application = application; - this.rc = rc; - this.rmContext = rmContext; - } + super(application, rc, rmContext); - protected boolean checkHeadroom(Resource clusterResource, - ResourceLimits currentResourceLimits, Resource required, - FiCaSchedulerNode node) { - // If headroom + currentReservation < required, we cannot allocate this - // require - Resource resourceCouldBeUnReserved = application.getCurrentReservation(); - if (!application.getCSLeafQueue().getReservationContinueLooking() - || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) { - // If we don't allow reservation continuous looking, OR we're looking at - // non-default node partition, we won't allow to unreserve before - // allocation. - resourceCouldBeUnReserved = Resources.none(); - } - return Resources.greaterThanOrEqual(rc, clusterResource, Resources.add( - currentResourceLimits.getHeadroom(), resourceCouldBeUnReserved), - required); + increaseContainerAllocator = + new IncreaseContainerAllocator(application, rc, rmContext); + regularContainerAllocator = + new RegularContainerAllocator(application, rc, rmContext); } - protected CSAssignment getCSAssignmentFromAllocateResult( - Resource clusterResource, ContainerAllocation result, - RMContainer rmContainer) { - // Handle skipped - boolean skipped = - (result.getAllocationState() == AllocationState.APP_SKIPPED); - CSAssignment assignment = new CSAssignment(skipped); - assignment.setApplication(application); - - // Handle excess reservation - assignment.setExcessReservation(result.getContainerToBeUnreserved()); - - // If we allocated something - if (Resources.greaterThan(rc, clusterResource, - result.getResourceToBeAllocated(), Resources.none())) { - Resource allocatedResource = result.getResourceToBeAllocated(); - Container updatedContainer = result.getUpdatedContainer(); - - assignment.setResource(allocatedResource); - assignment.setType(result.getContainerNodeType()); - - if (result.getAllocationState() == AllocationState.RESERVED) { - // This is a reserved container - LOG.info("Reserved container " + " application=" - + application.getApplicationId() + " resource=" + allocatedResource - + " queue=" + this.toString() + " cluster=" + clusterResource); - assignment.getAssignmentInformation().addReservationDetails( - updatedContainer.getId(), - application.getCSLeafQueue().getQueuePath()); - assignment.getAssignmentInformation().incrReservations(); - Resources.addTo(assignment.getAssignmentInformation().getReserved(), - allocatedResource); - } else if (result.getAllocationState() == AllocationState.ALLOCATED){ - // This is a new container - // Inform the ordering policy - LOG.info("assignedContainer" + " application attempt=" - + application.getApplicationAttemptId() + " container=" - + updatedContainer.getId() + " queue=" + this + " clusterResource=" - + clusterResource); - - application - .getCSLeafQueue() - .getOrderingPolicy() - .containerAllocated(application, - application.getRMContainer(updatedContainer.getId())); - - assignment.getAssignmentInformation().addAllocationDetails( - updatedContainer.getId(), - application.getCSLeafQueue().getQueuePath()); - assignment.getAssignmentInformation().incrAllocations(); - Resources.addTo(assignment.getAssignmentInformation().getAllocated(), - allocatedResource); - - if (rmContainer != null) { - assignment.setFulfilledReservation(true); - } + @Override + public CSAssignment assignContainers(Resource clusterResource, + FiCaSchedulerNode node, SchedulingMode schedulingMode, + ResourceLimits resourceLimits, RMContainer reservedContainer) { + if (reservedContainer != null) { + if (reservedContainer.getState() == RMContainerState.RESERVED) { + // It's a regular container + return regularContainerAllocator.assignContainers(clusterResource, + node, schedulingMode, resourceLimits, reservedContainer); + } else { + // It's a increase container + return increaseContainerAllocator.assignContainers(clusterResource, + node, schedulingMode, resourceLimits, reservedContainer); + } + } else { + /* + * Try to allocate increase container first, and if we failed to allocate + * anything, we will try to allocate regular container + */ + CSAssignment assign = + increaseContainerAllocator.assignContainers(clusterResource, node, + schedulingMode, resourceLimits, null); + if (Resources.greaterThan(rc, clusterResource, assign.getResource(), + Resources.none())) { + return assign; } + + return regularContainerAllocator.assignContainers(clusterResource, node, + schedulingMode, resourceLimits, null); } - - return assignment; } - - /** - * allocate needs to handle following stuffs: - * - *
      - *
    • Select request: Select a request to allocate. E.g. select a resource - * request based on requirement/priority/locality.
    • - *
    • Check if a given resource can be allocated based on resource - * availability
    • - *
    • Do allocation: this will decide/create allocated/reserved - * container, this will also update metrics
    • - *
    - */ - public abstract CSAssignment assignContainers(Resource clusterResource, - FiCaSchedulerNode node, SchedulingMode schedulingMode, - ResourceLimits resourceLimits, RMContainer reservedContainer); -} \ No newline at end of file + +} 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/capacity/allocator/IncreaseContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java new file mode 100644 index 0000000..9350adc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/IncreaseContainerAllocator.java @@ -0,0 +1,365 @@ +/** + * 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.scheduler.capacity.allocator; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +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.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; +import org.apache.hadoop.yarn.util.resource.ResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; + +public class IncreaseContainerAllocator extends AbstractContainerAllocator { + private static final Log LOG = + LogFactory.getLog(IncreaseContainerAllocator.class); + + public IncreaseContainerAllocator(FiCaSchedulerApp application, + ResourceCalculator rc, RMContext rmContext) { + super(application, rc, rmContext); + } + + /** + * Quick check if we can allocate anything here: + * We will not continue if: + * - Headroom doesn't support allocate minimumAllocation + * - + */ + private boolean checkHeadroom(Resource clusterResource, + ResourceLimits currentResourceLimits, Resource required) { + return Resources.greaterThanOrEqual(rc, clusterResource, + currentResourceLimits.getHeadroom(), required); + } + + private CSAssignment createReservedIncreasedCSAssignment( + SchedContainerChangeRequest request) { + CSAssignment assignment = + new CSAssignment(request.getDeltaCapacity(), NodeType.NODE_LOCAL, null, + application, false, false); + Resources.addTo(assignment.getAssignmentInformation().getReserved(), + request.getDeltaCapacity()); + assignment.getAssignmentInformation().incrReservations(); + assignment.getAssignmentInformation().addReservationDetails( + request.getContainerId(), application.getCSLeafQueue().getQueuePath()); + assignment.setIncreasedAllocation(true); + + LOG.info("Reserved increase container request:" + request.toString()); + + return assignment; + } + + private CSAssignment createSuccessfullyIncreasedCSAssignment( + SchedContainerChangeRequest request, boolean fromReservation) { + CSAssignment assignment = + new CSAssignment(request.getDeltaCapacity(), NodeType.NODE_LOCAL, null, + application, false, fromReservation); + Resources.addTo(assignment.getAssignmentInformation().getAllocated(), + request.getDeltaCapacity()); + assignment.getAssignmentInformation().incrAllocations(); + assignment.getAssignmentInformation().addAllocationDetails( + request.getContainerId(), application.getCSLeafQueue().getQueuePath()); + assignment.setIncreasedAllocation(true); + + // notify application + application + .getCSLeafQueue() + .getOrderingPolicy() + .containerAllocated(application, + application.getRMContainer(request.getContainerId())); + + LOG.info("Approved increase container request:" + request.toString() + + " fromReservation=" + fromReservation); + + return assignment; + } + + private CSAssignment allocateIncreaseRequestFromReservedContainer( + SchedulerNode node, Resource cluster, + SchedContainerChangeRequest increaseRequest) { + if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(), + node.getAvailableResource())) { + // OK, we can allocate this increase request + // Unreserve it first + application.unreserve(increaseRequest.getPriority(), + (FiCaSchedulerNode) node, increaseRequest.getRMContainer()); + + // Notify application + application.increaseContainer(increaseRequest); + + // Notify node + node.increaseContainer(increaseRequest.getContainerId(), + increaseRequest.getDeltaCapacity()); + + return createSuccessfullyIncreasedCSAssignment(increaseRequest, true); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to allocate reserved increase request:" + + increaseRequest.toString() + + ". There's no enough available resource"); + } + + // We still cannot allocate this container, will wait for next turn + return CSAssignment.SKIP_ASSIGNMENT; + } + } + + private CSAssignment allocateIncreaseRequest(FiCaSchedulerNode node, + Resource cluster, SchedContainerChangeRequest increaseRequest) { + if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(), + node.getAvailableResource())) { + // Notify node + node.increaseContainer(increaseRequest.getContainerId(), + increaseRequest.getDeltaCapacity()); + + // OK, we can allocate this increase request + // Notify application + application.increaseContainer(increaseRequest); + return createSuccessfullyIncreasedCSAssignment(increaseRequest, false); + } else { + boolean reservationSucceeded = + application.reserveIncreasedContainer(increaseRequest.getPriority(), + node, increaseRequest.getRMContainer(), + increaseRequest.getDeltaCapacity()); + + if (reservationSucceeded) { + // We cannot allocate this container, but since queue capacity / + // user-limit matches, we can reserve this container on this node. + return createReservedIncreasedCSAssignment(increaseRequest); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Reserve increase request=" + increaseRequest.toString() + + " failed. Skipping.."); + } + return CSAssignment.SKIP_ASSIGNMENT; + } + } + } + + @Override + public CSAssignment assignContainers(Resource clusterResource, + FiCaSchedulerNode node, SchedulingMode schedulingMode, + ResourceLimits resourceLimits, RMContainer reservedContainer) { + AppSchedulingInfo sinfo = application.getAppSchedulingInfo(); + NodeId nodeId = node.getNodeID(); + + if (reservedContainer == null) { + // Do we have increase request on this node? + if (!sinfo.hasIncreaseRequest(nodeId)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skip allocating increase request since we don't have any" + + " increase request on this node=" + node.getNodeID()); + } + + return CSAssignment.SKIP_ASSIGNMENT; + } + + // Check if we need to unreserve something, note that we don't support + // continuousReservationLooking now. TODO, need think more about how to + // support it. + boolean shouldUnreserve = + Resources.greaterThan(rc, clusterResource, + resourceLimits.getAmountNeededUnreserve(), Resources.none()); + + // Check if we can allocate minimum resource according to headroom + boolean cannotAllocateAnything = + !checkHeadroom(clusterResource, resourceLimits, rmContext + .getScheduler().getMinimumResourceCapability()); + + // Skip the app if we failed either of above check + if (cannotAllocateAnything || shouldUnreserve) { + if (LOG.isDebugEnabled()) { + if (shouldUnreserve) { + LOG.debug("Cannot continue since we have to unreserve some resource" + + ", now increase container allocation doesn't " + + "support continuous reservation looking.."); + } + if (cannotAllocateAnything) { + LOG.debug("We cannot allocate anything because of low headroom, " + + "headroom=" + resourceLimits.getHeadroom()); + } + } + + return CSAssignment.SKIP_ASSIGNMENT; + } + + CSAssignment assigned = null; + + /* + * Loop each priority, and containerId. Container priority is not + * equivalent to request priority, application master can run an important + * task on a less prioritized container. + * + * So behavior here is, we still try to increase container with higher + * priority, but will skip increase request and move to next increase + * request if queue-limit or user-limit aren't satisfied + */ + for (Priority priority : application.getPriorities()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Looking at increase request for application=" + + application.getApplicationAttemptId() + " priority=" + + priority); + } + + /* + * If we have multiple to-be-increased containers under same priority on + * a same host, we will try to increase earlier launched container + * first. And again - we will skip a request and move to next if it + * cannot be allocated. + */ + Map increaseRequestMap = + sinfo.getIncreaseRequests(nodeId, priority); + + // We don't have more increase request on this priority, skip.. + if (null == increaseRequestMap) { + if (LOG.isDebugEnabled()) { + LOG.debug("There's no increase request for " + + application.getApplicationAttemptId() + " priority=" + + priority); + } + continue; + } + Iterator> iter = + increaseRequestMap.entrySet().iterator(); + List toBeRemovedRequests = + new ArrayList<>(); + + while (iter.hasNext()) { + Entry entry = + iter.next(); + SchedContainerChangeRequest increaseRequest = + entry.getValue(); + + if (LOG.isDebugEnabled()) { + LOG.debug( + "Looking at increase request=" + increaseRequest.toString()); + } + + boolean headroomSatisifed = checkHeadroom(clusterResource, + resourceLimits, increaseRequest.getDeltaCapacity()); + if (!headroomSatisifed) { + // skip if doesn't satisfy headroom limit + if (LOG.isDebugEnabled()) { + LOG.debug(" Headroom is not satisfied, skip.."); + } + continue; + } + + RMContainer rmContainer = increaseRequest.getRMContainer(); + if (rmContainer.getContainerState() != ContainerState.RUNNING) { + // if the container is not running, we should remove the + // increaseRequest and continue; + if (LOG.isDebugEnabled()) { + LOG.debug(" Container is not running any more, skip..."); + } + toBeRemovedRequests.add(increaseRequest); + continue; + } + + if (!Resources.fitsIn(rc, clusterResource, + increaseRequest.getTargetCapacity(), node.getTotalResource())) { + // if the target capacity is more than what the node can offer, we + // will simply remove and skip it. + // The reason of doing check here instead of adding increase request + // to scheduler because node's resource could be updated after + // request added. + if (LOG.isDebugEnabled()) { + LOG.debug(" Target capacity is more than what node can offer," + + " node.resource=" + node.getTotalResource()); + } + toBeRemovedRequests.add(increaseRequest); + continue; + } + + // Try to allocate the increase request + assigned = + allocateIncreaseRequest(node, clusterResource, increaseRequest); + if (!assigned.getSkipped()) { + // When we don't skip this request, which means we either allocated + // OR reserved this request. We will break + break; + } + } + + // Remove invalid in request requests + if (!toBeRemovedRequests.isEmpty()) { + for (SchedContainerChangeRequest req : toBeRemovedRequests) { + sinfo.removeIncreaseRequest(req.getNodeId(), req.getPriority(), + req.getContainerId()); + } + } + + // We already allocated something + if (!assigned.getSkipped()) { + break; + } + } + + return assigned == null ? CSAssignment.SKIP_ASSIGNMENT : assigned; + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to allocate reserved increase container request.."); + } + + // We already reserved this increase container + SchedContainerChangeRequest request = + sinfo.getIncreaseRequest(nodeId, reservedContainer.getContainer() + .getPriority(), reservedContainer.getContainerId()); + + // We will cancel the reservation any of following happens + // - Container finished + // - No increase request needed + // - Target resource updated + if (null == request + || reservedContainer.getContainerState() != ContainerState.RUNNING + || (!Resources.equals(reservedContainer.getReservedResource(), + request.getDeltaCapacity()))) { + if (LOG.isDebugEnabled()) { + LOG.debug("We don't need reserved increase container request " + + "for container=" + reservedContainer.getContainerId() + + ". Unreserving and return..."); + } + + // We don't need this container now, just return excessive reservation + return new CSAssignment(application, reservedContainer); + } + + return allocateIncreaseRequestFromReservedContainer(node, clusterResource, + request); + } + } +} 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/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index dcb99ed..fd99d29 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -46,7 +46,7 @@ * Allocate normal (new) containers, considers locality/label, etc. Using * delayed scheduling mechanism to get better locality allocation. */ -public class RegularContainerAllocator extends ContainerAllocator { +public class RegularContainerAllocator extends AbstractContainerAllocator { private static final Log LOG = LogFactory.getLog(RegularContainerAllocator.class); private ResourceRequest lastResourceRequest = null; @@ -56,6 +56,25 @@ public RegularContainerAllocator(FiCaSchedulerApp application, super(application, rc, rmContext); } + private boolean checkHeadroom(Resource clusterResource, + ResourceLimits currentResourceLimits, Resource required, + FiCaSchedulerNode node) { + // If headroom + currentReservation < required, we cannot allocate this + // require + Resource resourceCouldBeUnReserved = application.getCurrentReservation(); + if (!application.getCSLeafQueue().getReservationContinueLooking() + || !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) { + // If we don't allow reservation continuous looking, OR we're looking at + // non-default node partition, we won't allow to unreserve before + // allocation. + resourceCouldBeUnReserved = Resources.none(); + } + return Resources.greaterThanOrEqual(rc, clusterResource, Resources.add( + currentResourceLimits.getHeadroom(), resourceCouldBeUnReserved), + required); + } + + private ContainerAllocation preCheckForNewContainer(Resource clusterResource, FiCaSchedulerNode node, SchedulingMode schedulingMode, ResourceLimits resourceLimits, Priority priority) { @@ -97,8 +116,9 @@ private ContainerAllocation preCheckForNewContainer(Resource clusterResource, // Is the node-label-expression of this offswitch resource request // matches the node's label? // If not match, jump to next priority. - if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(anyRequest, - node.getPartition(), schedulingMode)) { + if (!SchedulerUtils.checkResourceRequestMatchingNodePartition( + anyRequest.getNodeLabelExpression(), node.getPartition(), + schedulingMode)) { return ContainerAllocation.PRIORITY_SKIPPED; } @@ -388,8 +408,8 @@ private ContainerAllocation assignContainer(Resource clusterResource, } // check if the resource request can access the label - if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(request, - node.getPartition(), schedulingMode)) { + if (!SchedulerUtils.checkResourceRequestMatchingNodePartition( + request.getNodeLabelExpression(), node.getPartition(), schedulingMode)) { // this is a reserved container, but we cannot allocate it now according // to label not match. This can be caused by node label changed // We should un-reserve this 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/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index 300cba9..e97da24 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -32,6 +32,7 @@ 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.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; @@ -58,7 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.RegularContainerAllocator; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -83,7 +84,7 @@ private ResourceScheduler scheduler; - private ContainerAllocator containerAllocator; + private AbstractContainerAllocator containerAllocator; public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, String user, Queue queue, ActiveUsersManager activeUsersManager, @@ -118,7 +119,7 @@ public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, rc = scheduler.getResourceCalculator(); } - containerAllocator = new RegularContainerAllocator(this, rc, rmContext); + containerAllocator = new ContainerAllocator(this, rc, rmContext); } synchronized public boolean containerCompleted(RMContainer rmContainer, @@ -207,22 +208,24 @@ synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node, return rmContainer; } - public boolean unreserve(Priority priority, + public synchronized boolean unreserve(Priority priority, FiCaSchedulerNode node, RMContainer rmContainer) { + // Cancel increase request (if it has reserved increase request + rmContainer.cancelIncreaseReservation(); + // Done with the reservation? - if (unreserve(node, priority)) { + if (internalUnreserve(node, priority)) { node.unreserveResource(this); // Update reserved metrics queue.getMetrics().unreserveResource(getUser(), - rmContainer.getContainer().getResource()); + rmContainer.getReservedResource()); return true; } return false; } - @VisibleForTesting - public synchronized boolean unreserve(FiCaSchedulerNode node, Priority priority) { + private boolean internalUnreserve(FiCaSchedulerNode node, Priority priority) { Map reservedContainers = this.reservedContainers.get(priority); @@ -241,7 +244,7 @@ public synchronized boolean unreserve(FiCaSchedulerNode node, Priority priority) // Reset the re-reservation count resetReReservations(priority); - Resource resource = reservedContainer.getContainer().getResource(); + Resource resource = reservedContainer.getReservedResource(); this.attemptResourceUsage.decReserved(node.getPartition(), resource); LOG.info("Application " + getApplicationId() + " unreserved " @@ -311,13 +314,15 @@ public synchronized Allocation getAllocation(ResourceCalculator rc, ResourceRequest rr = ResourceRequest.newInstance( Priority.UNDEFINED, ResourceRequest.ANY, minimumAllocation, numCont); - ContainersAndNMTokensAllocation allocation = - pullNewlyAllocatedContainersAndNMTokens(); + List newlyAllocatedContainers = pullNewlyAllocatedContainers(); + List newlyIncreasedContainers = pullNewlyIncreasedContainers(); + List newlyDecreasedContainers = pullNewlyDecreasedContainers(); + List updatedNMTokens = pullUpdatedNMTokens(); Resource headroom = getHeadroom(); setApplicationHeadroomForMetrics(headroom); - return new Allocation(allocation.getContainerList(), headroom, null, - currentContPreemption, Collections.singletonList(rr), - allocation.getNMTokenList()); + return new Allocation(newlyAllocatedContainers, headroom, null, + currentContPreemption, Collections.singletonList(rr), updatedNMTokens, + newlyIncreasedContainers, newlyDecreasedContainers); } synchronized public NodeId getNodeIdToUnreserve(Priority priority, @@ -332,15 +337,23 @@ synchronized public NodeId getNodeIdToUnreserve(Priority priority, if ((reservedContainers != null) && (!reservedContainers.isEmpty())) { for (Map.Entry entry : reservedContainers.entrySet()) { NodeId nodeId = entry.getKey(); - Resource containerResource = entry.getValue().getContainer().getResource(); + RMContainer reservedContainer = entry.getValue(); + if (reservedContainer.hasIncreaseReservation()) { + // Currently, only regular container allocation supports continuous + // reservation looking, we don't support canceling increase request + // reservation when allocating regular container. + continue; + } + + Resource reservedResource = reservedContainer.getReservedResource(); // make sure we unreserve one with at least the same amount of // resources, otherwise could affect capacity limits - if (Resources.lessThanOrEqual(rc, clusterResource, - resourceNeedUnreserve, containerResource)) { + if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve, + reservedResource)) { if (LOG.isDebugEnabled()) { LOG.debug("unreserving node with reservation size: " - + containerResource + + reservedResource + " in order to allocate container with size: " + resourceNeedUnreserve); } return nodeId; @@ -374,6 +387,25 @@ public synchronized void transferStateFromPreviousAttempt( this.headroomProvider = ((FiCaSchedulerApp) appAttempt).getHeadroomProvider(); } + + public boolean reserveIncreasedContainer(Priority priority, + FiCaSchedulerNode node, + RMContainer rmContainer, Resource reservedResource) { + // Inform the application + if (super.reserveIncreasedContainer(node, priority, rmContainer, + reservedResource)) { + + queue.getMetrics().reserveResource(getUser(), reservedResource); + + // Update the node + node.reserveResource(this, priority, rmContainer); + + // Succeeded + return true; + } + + return false; + } public void reserve(Priority priority, FiCaSchedulerNode node, RMContainer rmContainer, Container 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/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index 3a39799..69654e8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -19,7 +19,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log; @@ -32,6 +39,7 @@ 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.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.QueueACL; @@ -68,7 +76,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; @@ -904,7 +913,9 @@ private synchronized void removeNode(RMNode rmNode) { @Override public Allocation allocate(ApplicationAttemptId appAttemptId, List ask, List release, - List blacklistAdditions, List blacklistRemovals) { + List blacklistAdditions, List blacklistRemovals, + List increaseRequests, + List decreaseRequests) { // Make sure this application exists FSAppAttempt application = getSchedulerApp(appAttemptId); @@ -963,18 +974,17 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, application.updateBlacklist(blacklistAdditions, blacklistRemovals); } - ContainersAndNMTokensAllocation allocation = - application.pullNewlyAllocatedContainersAndNMTokens(); - + List newlyAllocatedContainers = + application.pullNewlyAllocatedContainers(); // Record container allocation time - if (!(allocation.getContainerList().isEmpty())) { + if (!(newlyAllocatedContainers.isEmpty())) { application.recordContainerAllocationTime(getClock().getTime()); } Resource headroom = application.getHeadroom(); application.setApplicationHeadroomForMetrics(headroom); - return new Allocation(allocation.getContainerList(), headroom, - preemptionContainerIds, null, null, allocation.getNMTokenList()); + return new Allocation(newlyAllocatedContainers, headroom, + preemptionContainerIds, null, null, application.pullUpdatedNMTokens()); } } @@ -1706,4 +1716,11 @@ private String handleMoveToPlanQueue(String targetQueueName) { } return targetQueueName; } + + @Override + protected void decreaseContainer( + SchedContainerChangeRequest decreaseRequest, + SchedulerApplicationAttempt attempt) { + // TODO Auto-generated method stub + } } 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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 99760df..2ec2311 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -40,6 +40,7 @@ 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.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -76,7 +77,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; @@ -310,9 +311,11 @@ public synchronized void setRMContext(RMContext rmContext) { } @Override - public Allocation allocate( - ApplicationAttemptId applicationAttemptId, List ask, - List release, List blacklistAdditions, List blacklistRemovals) { + public Allocation allocate(ApplicationAttemptId applicationAttemptId, + List ask, List release, + List blacklistAdditions, List blacklistRemovals, + List increaseRequests, + List decreaseRequests) { FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId); if (application == null) { LOG.error("Calling allocate on removed " + @@ -364,12 +367,10 @@ public Allocation allocate( application.updateBlacklist(blacklistAdditions, blacklistRemovals); } - ContainersAndNMTokensAllocation allocation = - application.pullNewlyAllocatedContainersAndNMTokens(); Resource headroom = application.getHeadroom(); application.setApplicationHeadroomForMetrics(headroom); - return new Allocation(allocation.getContainerList(), headroom, null, - null, null, allocation.getNMTokenList()); + return new Allocation(application.pullNewlyAllocatedContainers(), + headroom, null, null, null, application.pullUpdatedNMTokens()); } } @@ -1005,4 +1006,12 @@ public synchronized boolean checkAccess(UserGroupInformation callerUGI, public Resource getUsedResource() { return usedResource; } + + @Override + protected void decreaseContainer( + SchedContainerChangeRequest decreaseRequest, + SchedulerApplicationAttempt attempt) { + // TODO Auto-generated method stub + + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java index e62f7d7..b536546 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java @@ -323,7 +323,7 @@ private synchronized void addResourceRequest( // Get resources from the ResourceManager Allocation allocation = resourceManager.getResourceScheduler().allocate( applicationAttemptId, new ArrayList(ask), - new ArrayList(), null, null); + new ArrayList(), null, null, null, null); System.out.println("-=======" + applicationAttemptId); System.out.println("----------" + resourceManager.getRMContext().getRMApps() .get(applicationId).getRMAppAttempt(applicationAttemptId)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java index 5660b78..c325a65 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java @@ -34,6 +34,7 @@ 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.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; @@ -235,6 +236,14 @@ public AllocateResponse allocate( releases, null); return allocate(req); } + + public AllocateResponse sendContainerResizingRequest( + List increaseRequests, + List decreaseRequests) throws Exception { + final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null, + null, increaseRequests, decreaseRequests); + return allocate(req); + } public AllocateResponse allocate(AllocateRequest allocateRequest) throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java index c917f79..4233cd4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java @@ -143,7 +143,7 @@ public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId, new HashMap>(1); ContainerStatus containerStatus = BuilderUtils.newContainerStatus( BuilderUtils.newContainerId(attemptId, containerId), containerState, - "Success", 0); + "Success", 0, BuilderUtils.newResource(memory, vCores)); ArrayList containerStatusList = new ArrayList(1); containerStatusList.add(containerStatus); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java index 53cb8d0..92f3edf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java @@ -19,11 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Set; import org.apache.hadoop.net.Node; 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.NodeId; import org.apache.hadoop.yarn.api.records.NodeState; @@ -231,6 +233,17 @@ public long getLastHealthReportTime() { } return CommonNodeLabelsManager.EMPTY_STRING_SET; } + + @Override + public void updateNodeHeartbeatResponseForContainersDecreasing( + NodeHeartbeatResponse response) { + + } + + @Override + public List pullNewlyIncreasedContainers() { + return Collections.emptyList(); + } }; private static RMNode buildRMNode(int rack, final Resource perNode, 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 e464401..7263b74 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 @@ -189,6 +189,19 @@ public void waitForState(ApplicationAttemptId attemptId, } } + public void waitForContainerState(ContainerId containerId, + RMContainerState state) throws Exception { + int timeoutSecs = 0; + RMContainer container = getResourceScheduler().getRMContainer(containerId); + while ((container == null || container.getState() != state) + && timeoutSecs++ < 40) { + System.out.println( + "Waiting for" + containerId + " state to be:" + state.name()); + Thread.sleep(200); + } + Assert.assertTrue(container.getState() == state); + } + public void waitForContainerAllocated(MockNM nm, ContainerId containerId) throws Exception { int timeoutSecs = 0; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java index d8d474e..b4ebf92 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java @@ -25,6 +25,8 @@ import java.util.List; import java.util.Map; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.junit.Assert; import org.apache.commons.logging.Log; @@ -191,7 +193,7 @@ synchronized public StartContainersResponse startContainers( ContainerStatus containerStatus = BuilderUtils.newContainerStatus(container.getId(), - ContainerState.NEW, "", -1000); + ContainerState.NEW, "", -1000, container.getResource()); applicationContainers.add(container); containerStatusMap.put(container, containerStatus); Resources.subtractFrom(available, tokenId.getResource()); @@ -295,7 +297,14 @@ synchronized public GetContainerStatusesResponse getContainerStatuses( return GetContainerStatusesResponse.newInstance(statuses, null); } - public static org.apache.hadoop.yarn.server.api.records.NodeStatus + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) + throws YarnException, IOException { + return null; + } + + public static org.apache.hadoop.yarn.server.api.records.NodeStatus createNodeStatus(NodeId nodeId, List containers) { RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); org.apache.hadoop.yarn.server.api.records.NodeStatus nodeStatus = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java index c7f0d0a..2787f1e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java @@ -40,6 +40,8 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest; @@ -122,6 +124,12 @@ public GetContainerStatusesResponse getContainerStatuses( return GetContainerStatusesResponse.newInstance(null, null); } + @Override + public IncreaseContainersResourceResponse increaseContainersResource(IncreaseContainersResourceRequest request) + throws YarnException { + return IncreaseContainersResourceResponse.newInstance(null, null); + } + public Credentials getContainerCredentials() throws IOException { Credentials credentials = new Credentials(); DataInputByteBuffer buf = new DataInputByteBuffer(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java index 6e08aeb..3fa377e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java @@ -231,7 +231,8 @@ protected Dispatcher createDispatcher() { ArrayList containerStatusList = new ArrayList(); containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0) - .getId(), ContainerState.RUNNING, "nothing", 0)); + .getId(), ContainerState.RUNNING, "nothing", 0, + conts.get(0).getResource())); containerStatuses.put(app.getApplicationId(), containerStatusList); NodeHeartbeatResponse resp = nm1.nodeHeartbeat(containerStatuses, true); @@ -244,7 +245,8 @@ protected Dispatcher createDispatcher() { containerStatuses.clear(); containerStatusList.clear(); containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0) - .getId(), ContainerState.RUNNING, "nothing", 0)); + .getId(), ContainerState.RUNNING, "nothing", 0, + conts.get(0).getResource())); containerStatuses.put(app.getApplicationId(), containerStatusList); resp = nm1.nodeHeartbeat(containerStatuses, true); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java index d4f8e93..2760705 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java @@ -32,6 +32,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest; +import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; @@ -126,6 +128,13 @@ public GetContainerStatusesResponse getContainerStatuses( GetContainerStatusesRequest request) throws YarnException { return null; } + + @Override + public IncreaseContainersResourceResponse increaseContainersResource( + IncreaseContainersResourceRequest request) + throws YarnException { + return null; + } } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java index 85d2515..8fa1ad2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java @@ -18,44 +18,51 @@ package org.apache.hadoop.yarn.server.resourcemanager; -import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; -import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; -import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; +import static java.lang.Thread.sleep; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; - +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl; -import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException; import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException; +import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.*; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +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.RMContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.utils.BuilderUtils; - +import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.Assert; - -import java.util.ArrayList; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static java.lang.Thread.sleep; public class TestApplicationMasterService { private static final Log LOG = LogFactory @@ -343,6 +350,92 @@ public void testAllocateAfterUnregister() throws Exception { alloc1Response = am1.schedule(); Assert.assertEquals(0, alloc1Response.getAllocatedContainers().size()); } + + @Test(timeout=60000) + public void testInvalidIncreaseDecreaseRequest() throws Exception { + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + MockRM rm = new MockRM(conf); + + try { + rm.start(); + + // Register node1 + MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB); + + // Submit an application + RMApp app1 = rm.submitApp(1024); + + // kick the scheduling + nm1.nodeHeartbeat(true); + RMAppAttempt attempt1 = app1.getCurrentAppAttempt(); + MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId()); + RegisterApplicationMasterResponse registerResponse = + am1.registerAppAttempt(); + + sentRMContainerLaunched(rm, + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1)); + + // Ask for a normal increase should be successfull + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest.newInstance( + ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), + Resources.createResource(2048))), null); + + // Target resource is negative, should fail + boolean exceptionCaught = false; + try { + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest.newInstance( + ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), + Resources.createResource(-1))), null); + } catch (InvalidResourceRequestException e) { + // This is expected + exceptionCaught = true; + } + Assert.assertTrue(exceptionCaught); + + // Target resource is more than maxAllocation, should fail + try { + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest.newInstance( + ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), + Resources + .add(registerResponse.getMaximumResourceCapability(), + Resources.createResource(1)))), null); + } catch (InvalidResourceRequestException e) { + // This is expected + exceptionCaught = true; + } + + Assert.assertTrue(exceptionCaught); + + // Contains multiple increase/decrease requests for same contaienrId + try { + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest.newInstance( + ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), + Resources + .add(registerResponse.getMaximumResourceCapability(), + Resources.createResource(1)))), Arrays.asList( + ContainerResourceChangeRequest.newInstance( + ContainerId.newContainerId(attempt1.getAppAttemptId(), 1), + Resources + .add(registerResponse.getMaximumResourceCapability(), + Resources.createResource(1))))); + } catch (InvalidResourceRequestException e) { + // This is expected + exceptionCaught = true; + } + + Assert.assertTrue(exceptionCaught); + } finally { + if (rm != null) { + rm.close(); + } + } + } private static class MyResourceManager extends MockRM { @@ -354,4 +447,15 @@ protected Dispatcher createDispatcher() { return new DrainDispatcher(); } } + + private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + RMContainer rmContainer = cs.getRMContainer(containerId); + if (rmContainer != null) { + rmContainer.handle( + new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED)); + } else { + Assert.fail("Cannot find RMContainer"); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index dc843b9..168280a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Test; @@ -331,11 +332,15 @@ public void testNMTokensRebindOnAMRestart() throws Exception { MockAM am2 = MockRM.launchAM(app1, rm1, nm1); RegisterApplicationMasterResponse registerResponse = am2.registerAppAttempt(); - rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); + rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.RUNNING); // check am2 get the nm token from am1. - Assert.assertEquals(expectedNMTokens, - registerResponse.getNMTokensFromPreviousAttempts()); + Assert.assertEquals(expectedNMTokens.size(), + registerResponse.getNMTokensFromPreviousAttempts().size()); + for (int i = 0; i < expectedNMTokens.size(); i++) { + Assert.assertTrue(expectedNMTokens.get(i) + .equals(registerResponse.getNMTokensFromPreviousAttempts().get(i))); + } // am2 allocate 1 container on nm2 containers = new ArrayList(); @@ -365,7 +370,7 @@ public void testNMTokensRebindOnAMRestart() throws Exception { // restart am MockAM am3 = MockRM.launchAM(app1, rm1, nm1); registerResponse = am3.registerAppAttempt(); - rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING); + rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.RUNNING); // check am3 get the NM token from both am1 and am2; List transferredTokens = registerResponse.getNMTokensFromPreviousAttempts(); @@ -430,7 +435,7 @@ protected Dispatcher createDispatcher() { ContainerStatus containerStatus = BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE, - "", ContainerExitStatus.DISKS_FAILED); + "", ContainerExitStatus.DISKS_FAILED, Resources.createResource(200)); currentNode.containerStatus(containerStatus); am1.waitForState(RMAppAttemptState.FAILED); rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java index 484a1b6..1f307aa 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/logaggregationstatus/TestRMAppLogAggregationStatus.java @@ -165,7 +165,7 @@ public void testLogAggregationStatus() throws Exception { node1ReportForApp.add(report1); node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus .newInstance(true, null, 0), new ArrayList(), null, - null, node1ReportForApp)); + null, node1ReportForApp, null)); List node2ReportForApp = new ArrayList(); @@ -177,7 +177,7 @@ public void testLogAggregationStatus() throws Exception { node2ReportForApp.add(report2); node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus .newInstance(true, null, 0), new ArrayList(), null, - null, node2ReportForApp)); + null, node2ReportForApp, null)); // node1 and node2 has updated its log aggregation status // verify that the log aggregation status for node1, node2 // has been changed @@ -215,7 +215,7 @@ public void testLogAggregationStatus() throws Exception { node1ReportForApp2.add(report1_2); node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus .newInstance(true, null, 0), new ArrayList(), null, - null, node1ReportForApp2)); + null, node1ReportForApp2, null)); // verify that the log aggregation status for node1 // has been changed @@ -284,7 +284,7 @@ public void testLogAggregationStatus() throws Exception { // 10 diagnostic messages/failure messages node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus .newInstance(true, null, 0), new ArrayList(), null, - null, node1ReportForApp3)); + null, node1ReportForApp3, null)); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); Assert.assertEquals(2, logAggregationStatus.size()); @@ -329,7 +329,7 @@ public void testLogAggregationStatus() throws Exception { node2ReportForApp2.add(report2_3); node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus .newInstance(true, null, 0), new ArrayList(), null, - null, node2ReportForApp2)); + null, node2ReportForApp2, null)); Assert.assertEquals(LogAggregationStatus.FAILED, rmApp.getLogAggregationStatusForAppReport()); logAggregationStatus = rmApp.getLogAggregationReportsForApp(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java index c8b6bd0..828e149 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java @@ -465,10 +465,9 @@ private void testAppAttemptScheduledState() { expectedAllocateCount = 1; } - assertEquals(expectedState, - applicationAttempt.getAppAttemptState()); - verify(scheduler, times(expectedAllocateCount)). - allocate(any(ApplicationAttemptId.class), + assertEquals(expectedState, applicationAttempt.getAppAttemptState()); + verify(scheduler, times(expectedAllocateCount)).allocate( + any(ApplicationAttemptId.class), any(List.class), any(List.class), any(List.class), any(List.class), any(List.class), any(List.class)); assertEquals(0,applicationAttempt.getJustFinishedContainers().size()); @@ -488,11 +487,9 @@ private void testAppAttemptAllocatedState(Container amContainer) { assertEquals(amContainer, applicationAttempt.getMasterContainer()); // Check events verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class)); - verify(scheduler, times(2)). - allocate( - any( - ApplicationAttemptId.class), any(List.class), any(List.class), - any(List.class), any(List.class)); + verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.class), + any(List.class), any(List.class), any(List.class), any(List.class), + any(List.class), any(List.class)); verify(nmTokenManager).clearNodeSetForAttempt( applicationAttempt.getAppAttemptId()); } @@ -641,13 +638,9 @@ private Container allocateApplicationAttempt() { Allocation allocation = mock(Allocation.class); when(allocation.getContainers()). thenReturn(Collections.singletonList(container)); - when( - scheduler.allocate( - any(ApplicationAttemptId.class), - any(List.class), - any(List.class), - any(List.class), - any(List.class))). + when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class), + any(List.class), any(List.class), any(List.class), any(List.class), + any(List.class))). thenReturn(allocation); RMContainer rmContainer = mock(RMContainerImpl.class); when(scheduler.getRMContainer(container.getId())). @@ -956,7 +949,8 @@ public void testAMCrashAtAllocated() { int exitCode = 123; ContainerStatus cs = BuilderUtils.newContainerStatus(amContainer.getId(), - ContainerState.COMPLETE, containerDiagMsg, exitCode); + ContainerState.COMPLETE, containerDiagMsg, exitCode, + amContainer.getResource()); NodeId anyNodeId = NodeId.newInstance("host", 1234); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), cs, anyNodeId)); @@ -980,7 +974,8 @@ public void testRunningToFailed() { String containerDiagMsg = "some error"; int exitCode = 123; ContainerStatus cs = BuilderUtils.newContainerStatus(amContainer.getId(), - ContainerState.COMPLETE, containerDiagMsg, exitCode); + ContainerState.COMPLETE, containerDiagMsg, exitCode, + amContainer.getResource()); ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); NodeId anyNodeId = NodeId.newInstance("host", 1234); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( @@ -992,7 +987,8 @@ public void testRunningToFailed() { applicationAttempt.getAppAttemptState()); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( - amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId)); + amContainer.getId(), ContainerState.COMPLETE, "", 0, + amContainer.getResource()), anyNodeId)); applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); assertEquals(RMAppAttemptState.FINAL_SAVING, @@ -1030,7 +1026,8 @@ public void testRunningToKilled() { NodeId anyNodeId = NodeId.newInstance("host", 1234); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( - amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId)); + amContainer.getId(), ContainerState.COMPLETE, "", 0, + amContainer.getResource()), anyNodeId)); applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); assertEquals(RMAppAttemptState.FINAL_SAVING, @@ -1207,7 +1204,8 @@ public void testFinishingToFinishing() { BuilderUtils.newContainerStatus( BuilderUtils.newContainerId( applicationAttempt.getAppAttemptId(), 42), - ContainerState.COMPLETE, "", 0), anyNodeId)); + ContainerState.COMPLETE, "", 0, + amContainer.getResource()), anyNodeId)); testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl, diagnostics); } @@ -1227,7 +1225,8 @@ public void testSuccessfulFinishingToFinished() { new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(amContainer.getId(), - ContainerState.COMPLETE, "", 0), anyNodeId)); + ContainerState.COMPLETE, "", 0, + amContainer.getResource()), anyNodeId)); testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl, diagnostics, 0, false); } @@ -1256,7 +1255,8 @@ public void testSuccessfulFinishingToFinished() { NodeId anyNodeId = NodeId.newInstance("host", 1234); applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( - amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId)); + amContainer.getId(), ContainerState.COMPLETE, "", 0, + amContainer.getResource()), anyNodeId)); assertEquals(RMAppAttemptState.FINAL_SAVING, applicationAttempt.getAppAttemptState()); // send attempt_saved @@ -1504,10 +1504,9 @@ public void testContainersCleanupForLastAttempt() { @Test public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() { YarnScheduler mockScheduler = mock(YarnScheduler.class); - when( - mockScheduler.allocate(any(ApplicationAttemptId.class), - any(List.class), any(List.class), any(List.class), any(List.class))) - .thenAnswer(new Answer() { + when(mockScheduler.allocate(any(ApplicationAttemptId.class), + any(List.class), any(List.class), any(List.class), any(List.class), + any(List.class), any(List.class))).thenAnswer(new Answer() { @SuppressWarnings("rawtypes") @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java index e4e2049..415e891 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; @@ -191,6 +190,10 @@ public void testExpireWhileRunning() { Container container = BuilderUtils.newContainer(containerId, nodeId, "host:3465", resource, priority, null); + + ConcurrentMap appMap = new ConcurrentHashMap<>(); + RMApp rmApp = mock(RMApp.class); + appMap.putIfAbsent(appId, rmApp); RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class); @@ -200,6 +203,7 @@ public void testExpireWhileRunning() { when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher); when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration()); + when(rmContext.getRMApps()).thenReturn(appMap); RMContainer rmContainer = new RMContainerImpl(container, appAttemptId, nodeId, "user", rmContext); @@ -235,11 +239,118 @@ public void testExpireWhileRunning() { rmContainer.handle(new RMContainerFinishedEvent(containerId, containerStatus, RMContainerEventType.EXPIRE)); drainDispatcher.await(); + assertEquals(RMContainerState.EXPIRED, rmContainer.getState()); + verify(writer, times(1)).containerFinished(any(RMContainer.class)); + verify(publisher, times(1)).containerFinished(any(RMContainer.class), + anyLong()); + } + + private void testExpireAfterIncreased(boolean acquired) { + /* + * Similar to previous test, a container is increased but not acquired by + * AM. In this case, if a container is expired, the container should be + * finished. + */ + DrainDispatcher drainDispatcher = new DrainDispatcher(); + EventHandler appAttemptEventHandler = + mock(EventHandler.class); + EventHandler generic = mock(EventHandler.class); + drainDispatcher.register(RMAppAttemptEventType.class, + appAttemptEventHandler); + drainDispatcher.register(RMNodeEventType.class, generic); + drainDispatcher.init(new YarnConfiguration()); + drainDispatcher.start(); + NodeId nodeId = BuilderUtils.newNodeId("host", 3425); + ApplicationId appId = BuilderUtils.newApplicationId(1, 1); + ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( + appId, 1); + ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1); + ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class); + + Resource resource = BuilderUtils.newResource(512, 1); + Priority priority = BuilderUtils.newPriority(5); + + Container container = BuilderUtils.newContainer(containerId, nodeId, + "host:3465", resource, priority, null); + + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); + SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class); + RMContext rmContext = mock(RMContext.class); + when(rmContext.getDispatcher()).thenReturn(drainDispatcher); + when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer); + when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); + when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher); + when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration()); + ConcurrentMap apps = + new ConcurrentHashMap(); + apps.put(appId, mock(RMApp.class)); + when(rmContext.getRMApps()).thenReturn(apps); + RMContainer rmContainer = new RMContainerImpl(container, appAttemptId, + nodeId, "user", rmContext); + + assertEquals(RMContainerState.NEW, rmContainer.getState()); + assertEquals(resource, rmContainer.getAllocatedResource()); + assertEquals(nodeId, rmContainer.getAllocatedNode()); + assertEquals(priority, rmContainer.getAllocatedPriority()); + verify(writer).containerStarted(any(RMContainer.class)); + verify(publisher).containerCreated(any(RMContainer.class), anyLong()); + + rmContainer.handle(new RMContainerEvent(containerId, + RMContainerEventType.START)); + drainDispatcher.await(); + assertEquals(RMContainerState.ALLOCATED, rmContainer.getState()); + + rmContainer.handle(new RMContainerEvent(containerId, + RMContainerEventType.ACQUIRED)); + drainDispatcher.await(); + assertEquals(RMContainerState.ACQUIRED, rmContainer.getState()); + + rmContainer.handle(new RMContainerEvent(containerId, + RMContainerEventType.LAUNCHED)); + drainDispatcher.await(); assertEquals(RMContainerState.RUNNING, rmContainer.getState()); - verify(writer, never()).containerFinished(any(RMContainer.class)); - verify(publisher, never()).containerFinished(any(RMContainer.class), + assertEquals( + "http://host:3465/node/containerlogs/container_1_0001_01_000001/user", + rmContainer.getLogURL()); + + // newResource is more than the old resource + Resource newResource = BuilderUtils.newResource(1024, 2); + rmContainer.handle(new RMContainerChangeResourceEvent(containerId, + newResource, true)); + + if (acquired) { + rmContainer + .handle(new RMContainerUpdatesAcquiredEvent(containerId, true)); + drainDispatcher.await(); + // status is still RUNNING since this is a increased container acquired by + // AM + assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + } + + // In RUNNING state. Verify EXPIRE and associated actions. + reset(appAttemptEventHandler); + ContainerStatus containerStatus = SchedulerUtils + .createAbnormalContainerStatus(containerId, + SchedulerUtils.EXPIRED_CONTAINER); + rmContainer.handle(new RMContainerFinishedEvent(containerId, + containerStatus, RMContainerEventType.EXPIRE)); + drainDispatcher.await(); + assertEquals(RMContainerState.EXPIRED, rmContainer.getState()); + + // Container will be finished only when it is acquired by AM after increase, + // we will only notify expirer when it is acquired by AM. + verify(writer, times(1)).containerFinished(any(RMContainer.class)); + verify(publisher, times(1)).containerFinished(any(RMContainer.class), anyLong()); } + + @Test + public void testExpireAfterContainerResourceIncreased() throws Exception { + // expire after increased and acquired by AM + testExpireAfterIncreased(true); + // expire after increased but not acquired by AM + testExpireAfterIncreased(false); + } @Test public void testExistenceOfResourceRequestInRMContainer() throws Exception { 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/TestCapacityScheduler.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/TestCapacityScheduler.java index 76a1351..d2fa2c1 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/TestCapacityScheduler.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/TestCapacityScheduler.java @@ -31,7 +31,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,6 +58,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.ContainerResourceChangeRequest; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; @@ -103,6 +103,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics; 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.RMContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; @@ -138,7 +140,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Mockito; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -677,11 +678,11 @@ public void testBlackListNodes() throws Exception { // Verify the blacklist can be updated independent of requesting containers cs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), - Collections.singletonList(host), null); + Collections.singletonList(host), null, null, null); Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); cs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, - Collections.singletonList(host)); + Collections.singletonList(host), null, null); Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); rm.stop(); } @@ -776,7 +777,7 @@ public void testAllocateReorder() throws Exception { cs.allocate(appAttemptId1, Collections.singletonList(r1), Collections.emptyList(), - null, null); + null, null, null, null); //And this will result in container assignment for app1 CapacityScheduler.schedule(cs); @@ -793,7 +794,7 @@ public void testAllocateReorder() throws Exception { cs.allocate(appAttemptId2, Collections.singletonList(r2), Collections.emptyList(), - null, null); + null, null, null, null); //In this case we do not perform container assignment because we want to //verify re-ordering based on the allocation alone @@ -869,7 +870,7 @@ public void testResourceOverCommit() throws Exception { // Check container can complete successfully in case of resource over-commitment. ContainerStatus containerStatus = BuilderUtils.newContainerStatus( - c1.getId(), ContainerState.COMPLETE, "", 0); + c1.getId(), ContainerState.COMPLETE, "", 0, c1.getResource()); nm1.containerStatus(containerStatus); int waitCount = 0; while (attempt1.getJustFinishedContainers().size() < 1 @@ -2909,7 +2910,7 @@ public void testApplicationHeadRoom() throws Exception { Allocation allocate = cs.allocate(appAttemptId, Collections. emptyList(), - Collections. emptyList(), null, null); + Collections. emptyList(), null, null, null, null); Assert.assertNotNull(attempt); @@ -2925,7 +2926,7 @@ public void testApplicationHeadRoom() throws Exception { allocate = cs.allocate(appAttemptId, Collections. emptyList(), - Collections. emptyList(), null, null); + Collections. emptyList(), null, null, null, null); // All resources should be sent as headroom Assert.assertEquals(newResource, allocate.getResourceLimit()); @@ -3086,7 +3087,107 @@ public void testAMLimitUsage() throws Exception { config.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS, DominantResourceCalculator.class.getName()); verifyAMLimitForLeafQueue(config); + } + + private FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm, + ApplicationId appId) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt(); + } + @Test + public void testPendingResourceUpdatedAccordingToIncreaseRequestChanges() + throws Exception { + Configuration conf = + TestUtils.getConfigurationWithQueueLabels(new Configuration(false)); + conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); + + final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + + MemoryRMStateStore memStore = new MemoryRMStateStore(); + memStore.init(conf); + MockRM rm = new MockRM(conf, memStore) { + protected RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + + rm.start(); + + MockNM nm1 = // label = "" + new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService()); + nm1.registerNode(); + + // Launch app1 in queue=a1 + RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), + "*", Resources.createResource(2 * GB), 2)), + null); + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + ContainerId containerId3 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 3); + Assert.assertTrue(rm.waitForState(nm1, containerId3, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them + am1.allocate(null, null); + sentRMContainerLaunched(rm, + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1L)); + sentRMContainerLaunched(rm, + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2L)); + sentRMContainerLaunched(rm, + ContainerId.newContainerId(am1.getApplicationAttemptId(), 3L)); + + // am1 asks to change its AM container from 1GB to 3GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(3 * GB))), + null); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm, app1.getApplicationId()); + + Assert.assertEquals(2 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + checkPendingResource(rm, "a1", 2 * GB, null); + checkPendingResource(rm, "a", 2 * GB, null); + checkPendingResource(rm, "root", 2 * GB, null); + + // am1 asks to change containerId2 (2G -> 3G) and containerId3 (2G -> 5G) + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId2, Resources.createResource(3 * GB)), + ContainerResourceChangeRequest + .newInstance(containerId3, Resources.createResource(5 * GB))), + null); + + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + checkPendingResource(rm, "a1", 6 * GB, null); + checkPendingResource(rm, "a", 6 * GB, null); + checkPendingResource(rm, "root", 6 * GB, null); + + // am1 asks to change containerId1 (1G->3G), containerId2 (2G -> 4G) and + // containerId3 (2G -> 2G) + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(3 * GB)), + ContainerResourceChangeRequest + .newInstance(containerId2, Resources.createResource(4 * GB)), + ContainerResourceChangeRequest + .newInstance(containerId3, Resources.createResource(2 * GB))), + null); + Assert.assertEquals(4 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + checkPendingResource(rm, "a1", 4 * GB, null); + checkPendingResource(rm, "a", 4 * GB, null); + checkPendingResource(rm, "root", 4 * GB, null); } private void verifyAMLimitForLeafQueue(CapacitySchedulerConfiguration config) @@ -3148,4 +3249,15 @@ private void setMaxAllocVcores(CapacitySchedulerConfiguration conf, + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES; conf.setInt(propName, maxAllocVcores); } + + private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + RMContainer rmContainer = cs.getRMContainer(containerId); + if (rmContainer != null) { + rmContainer.handle( + new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED)); + } else { + Assert.fail("Cannot find RMContainer"); + } + } } 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/TestChildQueueOrder.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/TestChildQueueOrder.java index 9dcab2e..88c7c13 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/TestChildQueueOrder.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/TestChildQueueOrder.java @@ -132,11 +132,11 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { final Resource allocatedResource = Resources.createResource(allocation); if (queue instanceof ParentQueue) { ((ParentQueue)queue).allocateResource(clusterResource, - allocatedResource, RMNodeLabelsManager.NO_LABEL); + allocatedResource, RMNodeLabelsManager.NO_LABEL, false); } else { FiCaSchedulerApp app1 = getMockApplication(0, ""); ((LeafQueue)queue).allocateResource(clusterResource, app1, - allocatedResource, null, null); + allocatedResource, null, null, false); } // Next call - nothing 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 769041b..b5b2222 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 @@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.LogAggregationContext; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -60,9 +59,6 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - public class TestContainerAllocation { @@ -199,13 +195,16 @@ public void testNormalContainerAllocationWhenDNSUnavailable() throws Exception{ // acquire the container. SecurityUtilTestHelper.setTokenServiceUseIp(true); - List containers = - am1.allocate(new ArrayList(), - new ArrayList()).getAllocatedContainers(); - // not able to fetch the container; - Assert.assertEquals(0, containers.size()); - - SecurityUtilTestHelper.setTokenServiceUseIp(false); + List containers; + try { + containers = + am1.allocate(new ArrayList(), + new ArrayList()).getAllocatedContainers(); + // not able to fetch the container; + Assert.assertEquals(0, containers.size()); + } finally { + SecurityUtilTestHelper.setTokenServiceUseIp(false); + } containers = am1.allocate(new ArrayList(), new ArrayList()).getAllocatedContainers(); @@ -315,21 +314,24 @@ protected RMSecretManagerService createRMSecretManagerService() { rm1.start(); MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000); - SecurityUtilTestHelper.setTokenServiceUseIp(true); - RMApp app1 = rm1.submitApp(200); - RMAppAttempt attempt = app1.getCurrentAppAttempt(); - nm1.nodeHeartbeat(true); - - // fetching am container will fail, keep retrying 5 times. - while (numRetries <= 5) { + RMApp app1; + try { + SecurityUtilTestHelper.setTokenServiceUseIp(true); + app1 = rm1.submitApp(200); + RMAppAttempt attempt = app1.getCurrentAppAttempt(); nm1.nodeHeartbeat(true); - Thread.sleep(1000); - Assert.assertEquals(RMAppAttemptState.SCHEDULED, - attempt.getAppAttemptState()); - System.out.println("Waiting for am container to be allocated."); - } - SecurityUtilTestHelper.setTokenServiceUseIp(false); + // fetching am container will fail, keep retrying 5 times. + while (numRetries <= 5) { + nm1.nodeHeartbeat(true); + Thread.sleep(1000); + Assert.assertEquals(RMAppAttemptState.SCHEDULED, + attempt.getAppAttemptState()); + System.out.println("Waiting for am container to be allocated."); + } + } finally { + SecurityUtilTestHelper.setTokenServiceUseIp(false); + } MockRM.launchAndRegisterAM(app1, rm1, nm1); } 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/TestContainerResizing.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/TestContainerResizing.java new file mode 100644 index 0000000..23283f6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java @@ -0,0 +1,963 @@ +/** +* 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.scheduler.capacity; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +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.ContainerResourceChangeRequest; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +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.RMContainerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestContainerResizing { + private final int GB = 1024; + + private YarnConfiguration conf; + + RMNodeLabelsManager mgr; + + @Before + public void setUp() throws Exception { + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + } + + @Test + public void testSimpleIncreaseContainer() throws Exception { + /** + * Application has a container running, and the node has enough available + * resource. Add a increase request to see if container will be increased + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + sentRMContainerLaunched(rm1, containerId1); + // am1 asks to change its AM container from 1GB to 3GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(3 * GB))), + null); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + checkPendingResource(rm1, "default", 2 * GB, null); + Assert.assertEquals(2 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + // Pending resource should be deducted + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + verifyContainerIncreased(am1.allocate(null, null), containerId1, 3 * GB); + verifyAvailableResourceOfSchedulerNode(rm1, nm1.getNodeId(), 17 * GB); + + rm1.close(); + } + + @Test + public void testSimpleDecreaseContainer() throws Exception { + /** + * Application has a container running, try to decrease the container and + * check queue's usage and container resource will be updated. + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 20 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(3 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + checkUsedResource(rm1, "default", 3 * GB, null); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + sentRMContainerLaunched(rm1, containerId1); + + // am1 asks to change its AM container from 1GB to 3GB + AllocateResponse response = am1.sendContainerResizingRequest(null, Arrays + .asList(ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(1 * GB)))); + + verifyContainerDecreased(response, containerId1, 1 * GB); + checkUsedResource(rm1, "default", 1 * GB, null); + Assert.assertEquals(1 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + // Check if decreased containers added to RMNode + RMNodeImpl rmNode = + (RMNodeImpl) rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + Collection decreasedContainers = + rmNode.getToBeDecreasedContainers(); + boolean rmNodeReceivedDecreaseContainer = false; + for (Container c : decreasedContainers) { + if (c.getId().equals(containerId1) + && c.getResource().equals(Resources.createResource(1 * GB))) { + rmNodeReceivedDecreaseContainer = true; + } + } + Assert.assertTrue(rmNodeReceivedDecreaseContainer); + + rm1.close(); + } + + @Test + public void testSimpleIncreaseRequestReservation() throws Exception { + /** + * Application has two containers running, try to increase one of then, node + * doesn't have enough resource, so the increase request will be reserved. + * Check resource usage after container reserved, finish a container, the + * reserved container should be allocated. + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); + MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*", + Resources.createResource(2 * GB), 1)), + null); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, containerId2, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them, and NM report RUNNING + am1.allocate(null, null); + sentRMContainerLaunched(rm1, containerId2); + + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + sentRMContainerLaunched(rm1, containerId1); + + + // am1 asks to change its AM container from 1GB to 3GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(7 * GB))), + null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1); + + /* Check reservation statuses */ + // Increase request should be reserved + Assert.assertTrue(rmContainer1.hasIncreaseReservation()); + Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory()); + Assert.assertFalse(app.getReservedContainers().isEmpty()); + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will not be changed since it's not satisfied + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 9 * GB, null); + Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + + // Complete one container and do another allocation + am1.allocate(null, Arrays.asList(containerId2)); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + // Now container should be increased + verifyContainerIncreased(am1.allocate(null, null), containerId1, 7 * GB); + + /* Check statuses after reservation satisfied */ + // Increase request should be unreserved + Assert.assertFalse(rmContainer1.hasIncreaseReservation()); + Assert.assertTrue(app.getReservedContainers().isEmpty()); + Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will be changed since it's satisfied + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 7 * GB, null); + Assert.assertEquals(7 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(7 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + verifyAvailableResourceOfSchedulerNode(rm1, nm1.getNodeId(), 1 * GB); + + rm1.close(); + } + + @Test + public void testExcessiveReservationWhenCancelIncreaseRequest() + throws Exception { + /** + * Application has two containers running, try to increase one of then, node + * doesn't have enough resource, so the increase request will be reserved. + * Check resource usage after container reserved, finish a container & + * cancel the increase request, reservation should be cancelled + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); + MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*", + Resources.createResource(2 * GB), 1)), + null); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, containerId2, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them, and NM report RUNNING + am1.allocate(null, null); + sentRMContainerLaunched(rm1, containerId2); + + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + sentRMContainerLaunched(rm1, containerId1); + + // am1 asks to change its AM container from 1GB to 3GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(7 * GB))), + null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1); + + /* Check reservation statuses */ + // Increase request should be reserved + Assert.assertTrue(rmContainer1.hasIncreaseReservation()); + Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory()); + Assert.assertFalse(app.getReservedContainers().isEmpty()); + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will not be changed since it's not satisfied + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 9 * GB, null); + Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + + // Complete one container and cancel increase request (via send a increase + // request, make target_capacity=existing_capacity) + am1.allocate(null, Arrays.asList(containerId2)); + // am1 asks to change its AM container from 1G to 1G (cancel the increase + // request actually) + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(1 * GB))), + null); + // Trigger a node heartbeat.. + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + /* Check statuses after reservation satisfied */ + // Increase request should be unreserved + Assert.assertTrue(app.getReservedContainers().isEmpty()); + Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + Assert.assertFalse(rmContainer1.hasIncreaseReservation()); + // Pending resource will be changed since it's satisfied + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 1 * GB, null); + Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(1 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + @Test + public void testExcessiveReservationWhenDecreaseSameContainer() + throws Exception { + /** + * Very similar to testExcessiveReservationWhenCancelIncreaseRequest, after + * the increase request reserved, it decreases the reserved container, + * container should be decreased and reservation will be cancelled + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); + MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(2 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*", + Resources.createResource(2 * GB), 1)), + null); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, containerId2, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them, and NM report RUNNING + am1.allocate(null, null); + sentRMContainerLaunched(rm1, containerId2); + + ContainerId containerId1 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 1); + sentRMContainerLaunched(rm1, containerId1); + + + // am1 asks to change its AM container from 2GB to 8GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(8 * GB))), + null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + RMContainer rmContainer1 = app.getLiveContainersMap().get(containerId1); + + /* Check reservation statuses */ + // Increase request should be reserved + Assert.assertTrue(rmContainer1.hasIncreaseReservation()); + Assert.assertEquals(6 * GB, rmContainer1.getReservedResource().getMemory()); + Assert.assertFalse(app.getReservedContainers().isEmpty()); + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will not be changed since it's not satisfied + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 10 * GB, null); + Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(4 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + + // Complete one container and cancel increase request (via send a increase + // request, make target_capacity=existing_capacity) + am1.allocate(null, Arrays.asList(containerId2)); + // am1 asks to change its AM container from 2G to 1G (decrease) + am1.sendContainerResizingRequest(null, Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId1, Resources.createResource(1 * GB)))); + // Trigger a node heartbeat.. + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + /* Check statuses after reservation satisfied */ + // Increase request should be unreserved + Assert.assertTrue(app.getReservedContainers().isEmpty()); + Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + Assert.assertFalse(rmContainer1.hasIncreaseReservation()); + // Pending resource will be changed since it's satisfied + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 1 * GB, null); + Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(1 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + @Test + public void testIncreaseContainerUnreservedWhenContainerCompleted() + throws Exception { + /** + * App has two containers on the same node (node.resource = 8G), container1 + * = 2G, container2 = 2G. App asks to increase container2 to 8G. + * + * So increase container request will be reserved. When app releases + * container2, reserved part should be released as well. + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); + MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*", + Resources.createResource(2 * GB), 1)), + null); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, containerId2, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them, and NM report RUNNING + am1.allocate(null, null); + sentRMContainerLaunched(rm1, containerId2); + rm1.waitForContainerState(containerId2, RMContainerState.RUNNING); + + // am1 asks to change its AM container from 2GB to 8GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId2, Resources.createResource(8 * GB))), + null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + RMContainer rmContainer2 = app.getLiveContainersMap().get(containerId2); + + /* Check reservation statuses */ + // Increase request should be reserved + Assert.assertTrue(rmContainer2.hasIncreaseReservation()); + Assert.assertEquals(6 * GB, rmContainer2.getReservedResource().getMemory()); + Assert.assertFalse(app.getReservedContainers().isEmpty()); + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will not be changed since it's not satisfied + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 9 * GB, null); + Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + + // Complete container2, container will be unreserved and completed + am1.allocate(null, Arrays.asList(containerId2)); + + /* Check statuses after reservation satisfied */ + // Increase request should be unreserved + Assert.assertTrue(app.getReservedContainers().isEmpty()); + Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + Assert.assertFalse(rmContainer2.hasIncreaseReservation()); + // Pending resource will be changed since it's satisfied + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 1 * GB, null); + Assert.assertEquals(1 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(1 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + @Test + public void testIncreaseContainerUnreservedWhenApplicationCompleted() + throws Exception { + /** + * Similar to testIncreaseContainerUnreservedWhenContainerCompleted, when + * application finishes, reserved increase container should be cancelled + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 8 * GB); + MockNM nm2 = rm1.registerNode("h2:1234", 8 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + + // Allocate two more containers + am1.allocate( + Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1), "*", + Resources.createResource(2 * GB), 1)), + null); + ContainerId containerId2 = + ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue( + rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED, + 10 * 1000)); + // Acquire them, and NM report RUNNING + am1.allocate(null, null); + sentRMContainerLaunched(rm1, containerId2); + + // am1 asks to change its AM container from 2GB to 8GB + am1.sendContainerResizingRequest(Arrays.asList( + ContainerResourceChangeRequest + .newInstance(containerId2, Resources.createResource(8 * GB))), + null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // NM1 do 1 heartbeats + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + + RMContainer rmContainer2 = app.getLiveContainersMap().get(containerId2); + + /* Check reservation statuses */ + // Increase request should be reserved + Assert.assertTrue(rmContainer2.hasIncreaseReservation()); + Assert.assertEquals(6 * GB, rmContainer2.getReservedResource().getMemory()); + Assert.assertFalse(app.getReservedContainers().isEmpty()); + Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + // Pending resource will not be changed since it's not satisfied + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 9 * GB, null); + Assert.assertEquals(9 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + + // Kill the application + cs.handle(new AppAttemptRemovedSchedulerEvent(am1.getApplicationAttemptId(), + RMAppAttemptState.KILLED, false)); + + /* Check statuses after reservation satisfied */ + // Increase request should be unreserved + Assert.assertTrue(app.getReservedContainers().isEmpty()); + Assert.assertNull(cs.getNode(nm1.getNodeId()).getReservedContainer()); + Assert.assertFalse(rmContainer2.hasIncreaseReservation()); + // Pending resource will be changed since it's satisfied + checkPendingResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 0 * GB, null); + Assert.assertEquals(0 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + private void allocateAndLaunchContainers(MockAM am, MockNM nm, MockRM rm, + int nContainer, int mem, int priority, int startContainerId) + throws Exception { + am.allocate(Arrays + .asList(ResourceRequest.newInstance(Priority.newInstance(priority), "*", + Resources.createResource(mem), nContainer)), + null); + ContainerId lastContainerId = ContainerId.newContainerId( + am.getApplicationAttemptId(), startContainerId + nContainer - 1); + Assert.assertTrue(rm.waitForState(nm, lastContainerId, + RMContainerState.ALLOCATED, 10 * 1000)); + // Acquire them, and NM report RUNNING + am.allocate(null, null); + + for (int cId = startContainerId; cId < startContainerId + + nContainer; cId++) { + sentRMContainerLaunched(rm, + ContainerId.newContainerId(am.getApplicationAttemptId(), cId)); + rm.waitForContainerState( + ContainerId.newContainerId(am.getApplicationAttemptId(), cId), + RMContainerState.RUNNING); + } + } + + @Test + public void testOrderOfIncreaseContainerRequestAllocation() + throws Exception { + /** + * There're multiple containers need to be increased, check container will + * be increase sorted by priority, if priority is same, smaller containerId + * container will get preferred + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + ApplicationAttemptId attemptId = am1.getApplicationAttemptId(); + + // Container 2, 3 (priority=3) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 3, 2); + + // Container 4, 5 (priority=2) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 2, 4); + + // Container 6, 7 (priority=4) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6); + + // am1 asks to change its container[2-7] from 1G to 2G + List increaseRequests = new ArrayList<>(); + for (int cId = 2; cId <= 7; cId++) { + ContainerId containerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), cId); + increaseRequests.add(ContainerResourceChangeRequest + .newInstance(containerId, Resources.createResource(2 * GB))); + } + am1.sendContainerResizingRequest(increaseRequests, null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // Get rmNode1 + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + + // assignContainer, container-4/5/2 increased (which has highest priority OR + // earlier allocated) + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + AllocateResponse allocateResponse = am1.allocate(null, null); + Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size()); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 4), 2 * GB); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 5), 2 * GB); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 2), 2 * GB); + + /* Check statuses after allocation */ + // There're still 3 pending increase requests + checkPendingResource(rm1, "default", 3 * GB, null); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 10 * GB, null); + Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(10 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + @Test + public void testIncreaseContainerRequestGetPreferrence() + throws Exception { + /** + * There're multiple containers need to be increased, and there're several + * container allocation request, scheduler will try to increase container + * before allocate new containers + */ + MockRM rm1 = new MockRM() { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.start(); + MockNM nm1 = rm1.registerNode("h1:1234", 10 * GB); + + // app1 -> a1 + RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default"); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + FiCaSchedulerApp app = getFiCaSchedulerApp(rm1, app1.getApplicationId()); + ApplicationAttemptId attemptId = am1.getApplicationAttemptId(); + + // Container 2, 3 (priority=3) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 3, 2); + + // Container 4, 5 (priority=2) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 2, 4); + + // Container 6, 7 (priority=4) + allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6); + + // am1 asks to change its container[2-7] from 1G to 2G + List increaseRequests = new ArrayList<>(); + for (int cId = 2; cId <= 7; cId++) { + ContainerId containerId = + ContainerId.newContainerId(am1.getApplicationAttemptId(), cId); + increaseRequests.add(ContainerResourceChangeRequest + .newInstance(containerId, Resources.createResource(2 * GB))); + } + am1.sendContainerResizingRequest(increaseRequests, null); + + checkPendingResource(rm1, "default", 6 * GB, null); + Assert.assertEquals(6 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + + // Get rmNode1 + CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler(); + RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId()); + + // assignContainer, container-4/5/2 increased (which has highest priority OR + // earlier allocated) + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + AllocateResponse allocateResponse = am1.allocate(null, null); + Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size()); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 4), 2 * GB); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 5), 2 * GB); + verifyContainerIncreased(allocateResponse, + ContainerId.newContainerId(attemptId, 2), 2 * GB); + + /* Check statuses after allocation */ + // There're still 3 pending increase requests + checkPendingResource(rm1, "default", 3 * GB, null); + Assert.assertEquals(3 * GB, + app.getAppAttemptResourceUsage().getPending().getMemory()); + // Queue/user/application's usage will be updated + checkUsedResource(rm1, "default", 10 * GB, null); + Assert.assertEquals(10 * GB, ((LeafQueue) cs.getQueue("default")) + .getUser("user").getUsed().getMemory()); + Assert.assertEquals(0 * GB, + app.getAppAttemptResourceUsage().getReserved().getMemory()); + Assert.assertEquals(10 * GB, + app.getAppAttemptResourceUsage().getUsed().getMemory()); + + rm1.close(); + } + + private void checkPendingResource(MockRM rm, String queueName, int memory, + String label) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + CSQueue queue = cs.getQueue(queueName); + Assert.assertEquals(memory, + queue.getQueueResourceUsage() + .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label) + .getMemory()); + } + + private void checkUsedResource(MockRM rm, String queueName, int memory, + String label) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + CSQueue queue = cs.getQueue(queueName); + Assert.assertEquals(memory, + queue.getQueueResourceUsage() + .getUsed(label == null ? RMNodeLabelsManager.NO_LABEL : label) + .getMemory()); + } + + private void verifyContainerIncreased(AllocateResponse response, + ContainerId containerId, int mem) { + List increasedContainers = response.getIncreasedContainers(); + boolean found = false; + for (Container c : increasedContainers) { + if (c.getId().equals(containerId)) { + found = true; + Assert.assertEquals(mem, c.getResource().getMemory()); + } + } + if (!found) { + Assert.fail("Container not increased: containerId=" + containerId); + } + } + + private void verifyContainerDecreased(AllocateResponse response, + ContainerId containerId, int mem) { + List decreasedContainers = response.getDecreasedContainers(); + boolean found = false; + for (Container c : decreasedContainers) { + if (c.getId().equals(containerId)) { + found = true; + Assert.assertEquals(mem, c.getResource().getMemory()); + } + } + if (!found) { + Assert.fail("Container not decreased: containerId=" + containerId); + } + } + + private void sentRMContainerLaunched(MockRM rm, ContainerId containerId) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + RMContainer rmContainer = cs.getRMContainer(containerId); + if (rmContainer != null) { + rmContainer.handle( + new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED)); + } else { + Assert.fail("Cannot find RMContainer"); + } + } + + private void verifyAvailableResourceOfSchedulerNode(MockRM rm, NodeId nodeId, + int expectedMemory) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + SchedulerNode node = cs.getNode(nodeId); + Assert + .assertEquals(expectedMemory, node.getAvailableResource().getMemory()); + } + + private FiCaSchedulerApp getFiCaSchedulerApp(MockRM rm, + ApplicationId appId) { + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + return cs.getSchedulerApplications().get(appId).getCurrentAppAttempt(); + } +} 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/TestLeafQueue.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/TestLeafQueue.java index fe8be06..b85c697 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/TestLeafQueue.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/TestLeafQueue.java @@ -770,9 +770,9 @@ public void testComputeUserLimitAndSetHeadroom(){ qb.finishApplication(app_0.getApplicationId(), user_0); qb.finishApplication(app_2.getApplicationId(), user_1); qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority), - null, null); + null, null, false); qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority), - null, null); + null, null, false); qb.setUserLimit(50); qb.setUserLimitFactor(1); 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/TestParentQueue.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/TestParentQueue.java index ef35093..4a815f5 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/TestParentQueue.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/TestParentQueue.java @@ -144,11 +144,11 @@ public CSAssignment answer(InvocationOnMock invocation) throws Throwable { final Resource allocatedResource = Resources.createResource(allocation); if (queue instanceof ParentQueue) { ((ParentQueue)queue).allocateResource(clusterResource, - allocatedResource, RMNodeLabelsManager.NO_LABEL); + allocatedResource, RMNodeLabelsManager.NO_LABEL, false); } else { FiCaSchedulerApp app1 = getMockApplication(0, ""); ((LeafQueue)queue).allocateResource(clusterResource, app1, - allocatedResource, null, null); + allocatedResource, null, null, false); } // Next call - nothing 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/TestReservations.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/TestReservations.java index 6a0b11b..884de2a 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/TestReservations.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/TestReservations.java @@ -60,6 +60,9 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.junit.Before; import org.junit.Test; @@ -482,6 +485,8 @@ public void testReservationNoContinueLook() throws Exception { @Test public void testAssignContainersNeedToUnreserve() throws Exception { // Test that we now unreserve and use a node that has space + Logger rootLogger = LogManager.getRootLogger(); + rootLogger.setLevel(Level.DEBUG); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); setup(csConf); @@ -593,7 +598,7 @@ public void testAssignContainersNeedToUnreserve() throws Exception { assertEquals(2, app_0.getTotalRequiredResources(priorityReduce)); // could allocate but told need to unreserve first - CSAssignment csAssignment = a.assignContainers(clusterResource, node_1, + a.assignContainers(clusterResource, node_1, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY); assertEquals(13 * GB, a.getUsedResources().getMemory()); assertEquals(13 * GB, app_0.getCurrentConsumption().getMemory()); @@ -664,7 +669,7 @@ public void testGetAppToUnreserve() throws Exception { // no reserved containers - reserve then unreserve app_0.reserve(node_0, priorityMap, rmContainer_1, container_1); - app_0.unreserve(node_0, priorityMap); + app_0.unreserve(priorityMap, node_0, rmContainer_1); unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability, cs.getResourceCalculator(), clusterResource); assertEquals(null, unreserveId); 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/fair/FairSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java index 1c9801d..3af3424 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java @@ -163,7 +163,7 @@ protected ApplicationAttemptId createSchedulingRequest( resourceManager.getRMContext().getRMApps() .put(id.getApplicationId(), rmApp); - scheduler.allocate(id, ask, new ArrayList(), null, null); + scheduler.allocate(id, ask, new ArrayList(), null, null, null, null); return id; } @@ -189,7 +189,7 @@ protected ApplicationAttemptId createSchedulingRequest(String queueId, resourceManager.getRMContext().getRMApps() .put(id.getApplicationId(), rmApp); - scheduler.allocate(id, ask, new ArrayList(), null, null); + scheduler.allocate(id, ask, new ArrayList(), null, null, null, null); return id; } @@ -211,7 +211,7 @@ protected void createSchedulingRequestExistingApplication( ResourceRequest request, ApplicationAttemptId attId) { List ask = new ArrayList(); ask.add(request); - scheduler.allocate(attId, ask, new ArrayList(), null, null); + scheduler.allocate(attId, ask, new ArrayList(), null, null, null, null); } protected void createApplicationWithAMResource(ApplicationAttemptId attId, 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/fair/TestContinuousScheduling.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java index 53382de..65c80a6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestContinuousScheduling.java @@ -99,7 +99,7 @@ public void testSchedulingDelay() throws InterruptedException { List ask = new ArrayList<>(); ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true)); scheduler.allocate( - appAttemptId, ask, new ArrayList(), null, null); + appAttemptId, ask, new ArrayList(), null, null, null, null); FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId); // Advance time and let continuous scheduling kick in 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/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java index a02cf18..73e09de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java @@ -1375,7 +1375,7 @@ public void testQueueDemandCalculation() throws Exception { ResourceRequest request1 = createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true); ask1.add(request1); - scheduler.allocate(id11, ask1, new ArrayList(), null, null); + scheduler.allocate(id11, ask1, new ArrayList(), null, null, null, null); // Second ask, queue2 requests 1 large + (2 * minReqSize) List ask2 = new ArrayList(); @@ -1385,14 +1385,14 @@ public void testQueueDemandCalculation() throws Exception { false); ask2.add(request2); ask2.add(request3); - scheduler.allocate(id21, ask2, new ArrayList(), null, null); + scheduler.allocate(id21, ask2, new ArrayList(), null, null, null, null); // Third ask, queue2 requests 1 large List ask3 = new ArrayList(); ResourceRequest request4 = createResourceRequest(2 * minReqSize, ResourceRequest.ANY, 1, 1, true); ask3.add(request4); - scheduler.allocate(id22, ask3, new ArrayList(), null, null); + scheduler.allocate(id22, ask3, new ArrayList(), null, null, null, null); scheduler.update(); @@ -2714,7 +2714,7 @@ public void testReservationWhileMultiplePriorities() throws IOException { // Complete container scheduler.allocate(attId, new ArrayList(), - Arrays.asList(containerId), null, null); + Arrays.asList(containerId), null, null, null, null); assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB()); assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores()); @@ -2806,7 +2806,7 @@ public void testMultipleNodesSingleRackRequest() throws Exception { asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true)); scheduler.allocate(attemptId, asks, new ArrayList(), null, - null); + null, null, null); // node 1 checks in scheduler.update(); @@ -3202,7 +3202,7 @@ public void testCancelStrictLocality() throws IOException { createResourceRequest(1024, node1.getHostName(), 1, 0, true), createResourceRequest(1024, "rack1", 1, 0, true), createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true)); - scheduler.allocate(attId1, update, new ArrayList(), null, null); + scheduler.allocate(attId1, update, new ArrayList(), null, null, null, null); // then node2 should get the container scheduler.handle(node2UpdateEvent); @@ -3249,7 +3249,7 @@ public void testReservationsStrictLocality() throws IOException { anyRequest = createResourceRequest(1024, ResourceRequest.ANY, 1, 1, false); scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest), - new ArrayList(), null, null); + new ArrayList(), null, null, null, null); scheduler.handle(nodeUpdateEvent); assertEquals(0, app.getReservedContainers().size()); @@ -4251,7 +4251,7 @@ public void testContinuousScheduling() throws Exception { ResourceRequest request = createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true); ask.add(request); - scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null); + scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null, null, null); // waiting for continuous_scheduler_sleep_time // at least one pass @@ -4271,7 +4271,7 @@ public void testContinuousScheduling() throws Exception { ask.clear(); ask.add(request); scheduler.stop(); - scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null); + scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null, null, null); scheduler.continuousSchedulingAttempt(); Assert.assertEquals(2048, app.getCurrentConsumption().getMemory()); Assert.assertEquals(2, app.getCurrentConsumption().getVirtualCores()); @@ -4371,7 +4371,7 @@ public void testSchedulingOnRemovedNode() throws Exception { ask1.add(request1); scheduler.allocate(id11, ask1, new ArrayList(), null, - null); + null, null, null); String hostName = "127.0.0.1"; RMNode node1 = MockNodes.newNodeInfo(1, @@ -4503,7 +4503,7 @@ public void testRecoverRequestAfterPreemption() throws Exception { List containers = scheduler.allocate(appAttemptId, Collections. emptyList(), - Collections. emptyList(), null, null).getContainers(); + Collections. emptyList(), null, null, null, null).getContainers(); // Now with updated ResourceRequest, a container is allocated for AM. Assert.assertTrue(containers.size() == 1); @@ -4532,11 +4532,11 @@ public void testBlacklistNodes() throws Exception { // Verify the blacklist can be updated independent of requesting containers scheduler.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), - Collections.singletonList(host), null); + Collections.singletonList(host), null, null, null); assertTrue(app.isBlacklisted(host)); scheduler.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, - Collections.singletonList(host)); + Collections.singletonList(host), null, null); assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host)); List update = Arrays.asList( @@ -4545,7 +4545,7 @@ public void testBlacklistNodes() throws Exception { // Verify a container does not actually get placed on the blacklisted host scheduler.allocate(appAttemptId, update, Collections.emptyList(), - Collections.singletonList(host), null); + Collections.singletonList(host), null, null, null); assertTrue(app.isBlacklisted(host)); scheduler.update(); scheduler.handle(updateEvent); @@ -4555,7 +4555,7 @@ public void testBlacklistNodes() throws Exception { // Verify a container gets placed on the empty blacklist scheduler.allocate(appAttemptId, update, Collections.emptyList(), null, - Collections.singletonList(host)); + Collections.singletonList(host), null, null); assertFalse(app.isBlacklisted(host)); createSchedulingRequest(GB, "root.default", "user", 1); scheduler.update(); 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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index 5b5c5ed..83ba2d5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -263,7 +263,7 @@ public void testNodeLocalAssignment() throws Exception { ask.add(nodeLocal); ask.add(rackLocal); ask.add(any); - scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null); + scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null, null, null); NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0); @@ -365,7 +365,7 @@ public void testUpdateResourceOnNode() throws Exception { ask.add(nodeLocal); ask.add(rackLocal); ask.add(any); - scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null); + scheduler.allocate(appAttemptId, ask, new ArrayList(), null, null, null, null); // Before the node update event, there are one local request Assert.assertEquals(1, nodeLocal.getNumContainers()); @@ -746,7 +746,7 @@ public void testFifoScheduling() throws Exception { Assert.assertEquals(GB, c1.getResource().getMemory()); ContainerStatus containerStatus = BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE, - "", 0); + "", 0, c1.getResource()); nm1.containerStatus(containerStatus); int waitCount = 0; while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) { @@ -941,7 +941,7 @@ public void testBlackListNodes() throws Exception { ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1)); fs.allocate(appAttemptId1, ask1, emptyId, - Collections.singletonList(host_1_0), null); + Collections.singletonList(host_1_0), null, null, null); // Trigger container assignment fs.handle(new NodeUpdateSchedulerEvent(n3)); @@ -949,14 +949,14 @@ public void testBlackListNodes() throws Exception { // Get the allocation for the application and verify no allocation on // blacklist node Allocation allocation1 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation1", 0, allocation1.getContainers().size()); // verify host_1_1 can get allocated as not in blacklist fs.handle(new NodeUpdateSchedulerEvent(n4)); Allocation allocation2 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation2", 1, allocation2.getContainers().size()); List containerList = allocation2.getContainers(); for (Container container : containerList) { @@ -971,29 +971,29 @@ public void testBlackListNodes() throws Exception { ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1)); fs.allocate(appAttemptId1, ask2, emptyId, - Collections.singletonList("rack0"), null); + Collections.singletonList("rack0"), null, null, null); // verify n1 is not qualified to be allocated fs.handle(new NodeUpdateSchedulerEvent(n1)); Allocation allocation3 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation3", 0, allocation3.getContainers().size()); // verify n2 is not qualified to be allocated fs.handle(new NodeUpdateSchedulerEvent(n2)); Allocation allocation4 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation4", 0, allocation4.getContainers().size()); // verify n3 is not qualified to be allocated fs.handle(new NodeUpdateSchedulerEvent(n3)); Allocation allocation5 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation5", 0, allocation5.getContainers().size()); fs.handle(new NodeUpdateSchedulerEvent(n4)); Allocation allocation6 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("allocation6", 1, allocation6.getContainers().size()); containerList = allocation6.getContainers(); @@ -1052,25 +1052,25 @@ public void testHeadroom() throws Exception { List ask1 = new ArrayList(); ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1)); - fs.allocate(appAttemptId1, ask1, emptyId, null, null); + fs.allocate(appAttemptId1, ask1, emptyId, null, null, null, null); // Ask for a 2 GB container for app 2 List ask2 = new ArrayList(); ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1)); - fs.allocate(appAttemptId2, ask2, emptyId, null, null); + fs.allocate(appAttemptId2, ask2, emptyId, null, null, null, null); // Trigger container assignment fs.handle(new NodeUpdateSchedulerEvent(n1)); // Get the allocation for the applications and verify headroom Allocation allocation1 = - fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("Allocation headroom", 1 * GB, allocation1 .getResourceLimit().getMemory()); Allocation allocation2 = - fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null); + fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null, null, null); Assert.assertEquals("Allocation headroom", 1 * GB, allocation2 .getResourceLimit().getMemory()); @@ -1141,7 +1141,7 @@ public void testResourceOverCommit() throws Exception { // over-commitment. ContainerStatus containerStatus = BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE, - "", 0); + "", 0, c1.getResource()); nm1.containerStatus(containerStatus); int waitCount = 0; while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java index 5dfd092..4488ad6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java @@ -171,7 +171,8 @@ public void testTokenExpiry() throws Exception { ContainerStatus containerStatus = BuilderUtils.newContainerStatus(attempt.getMasterContainer().getId(), ContainerState.COMPLETE, - "AM Container Finished", 0); + "AM Container Finished", 0, + attempt.getMasterContainer().getResource()); rm.getRMContext() .getDispatcher() .getEventHandler()