YARN-1651. CapacityScheduler side changes to support container resize. Contributed by Wangda Tan
(cherry picked from commit 89cab1ba5f
)
Conflicts:
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java
This commit is contained in:
parent
d592aa115b
commit
d458805690
|
@ -97,6 +97,7 @@ 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.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;
|
||||
|
@ -1633,8 +1634,10 @@ public class TestRMContainerAllocator {
|
|||
@Override
|
||||
public synchronized Allocation allocate(
|
||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||
List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||
List<ContainerId> release, List<String> blacklistAdditions,
|
||||
List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||
for (ResourceRequest req : ask) {
|
||||
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
||||
|
@ -1648,8 +1651,8 @@ public class TestRMContainerAllocator {
|
|||
lastBlacklistAdditions = blacklistAdditions;
|
||||
lastBlacklistRemovals = blacklistRemovals;
|
||||
return super.allocate(
|
||||
applicationAttemptId, askCopy, release,
|
||||
blacklistAdditions, blacklistRemovals);
|
||||
applicationAttemptId, askCopy, release, blacklistAdditions,
|
||||
blacklistRemovals, increaseRequests, decreaseRequests);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1669,8 +1672,10 @@ public class TestRMContainerAllocator {
|
|||
@Override
|
||||
public synchronized Allocation allocate(
|
||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||
List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||
List<ContainerId> release, List<String> blacklistAdditions,
|
||||
List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequest,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||
for (ResourceRequest req : ask) {
|
||||
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
||||
|
@ -1681,7 +1686,7 @@ public class TestRMContainerAllocator {
|
|||
SecurityUtil.setTokenServiceUseIp(false);
|
||||
Allocation normalAlloc = super.allocate(
|
||||
applicationAttemptId, askCopy, release,
|
||||
blacklistAdditions, blacklistRemovals);
|
||||
blacklistAdditions, blacklistRemovals, null, null);
|
||||
List<Container> containers = normalAlloc.getContainers();
|
||||
if(containers.size() > 0) {
|
||||
// allocate excess container
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|||
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 class NodeInfo {
|
|||
public Set<String> getNodeLabels() {
|
||||
return RMNodeLabelsManager.EMPTY_STRING_SET;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateNodeHeartbeatResponseForContainersDecreasing(
|
||||
NodeHeartbeatResponse response) {
|
||||
// TODO Auto-generated method stub
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Container> pullNewlyIncreasedContainers() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static RMNode newNodeInfo(String rackName, String hostName,
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|||
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 class RMNodeWrapper implements RMNode {
|
|||
public Set<String> getNodeLabels() {
|
||||
return RMNodeLabelsManager.EMPTY_STRING_SET;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateNodeHeartbeatResponseForContainersDecreasing(
|
||||
NodeHeartbeatResponse response) {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Container> pullNewlyIncreasedContainers() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
|||
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.ResourceScheduler
|
|||
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;
|
||||
|
@ -203,15 +205,16 @@ public class ResourceSchedulerWrapper
|
|||
|
||||
@Override
|
||||
public Allocation allocate(ApplicationAttemptId attemptId,
|
||||
List<ResourceRequest> resourceRequests,
|
||||
List<ContainerId> containerIds,
|
||||
List<String> strings, List<String> strings2) {
|
||||
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
|
||||
List<String> strings, List<String> strings2,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> 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();
|
||||
|
@ -225,7 +228,7 @@ public class ResourceSchedulerWrapper
|
|||
}
|
||||
} else {
|
||||
return scheduler.allocate(attemptId,
|
||||
resourceRequests, containerIds, strings, strings2);
|
||||
resourceRequests, containerIds, strings, strings2, null, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -960,5 +963,13 @@ public class ResourceSchedulerWrapper
|
|||
return Priority.newInstance(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decreaseContainer(
|
||||
SchedContainerChangeRequest decreaseRequest,
|
||||
SchedulerApplicationAttempt attempt) {
|
||||
// TODO Auto-generated method stub
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -165,6 +165,9 @@ Release 2.8.0 - UNRELEASED
|
|||
|
||||
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
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|||
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 class TestAMRMClientOnRMRestart {
|
|||
public synchronized Allocation allocate(
|
||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||
List<ContainerId> release, List<String> blacklistAdditions,
|
||||
List<String> blacklistRemovals) {
|
||||
List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||
for (ResourceRequest req : ask) {
|
||||
ResourceRequest reqCopy =
|
||||
|
@ -539,7 +542,8 @@ public class TestAMRMClientOnRMRestart {
|
|||
lastBlacklistAdditions = blacklistAdditions;
|
||||
lastBlacklistRemovals = blacklistRemovals;
|
||||
return super.allocate(applicationAttemptId, askCopy, release,
|
||||
blacklistAdditions, blacklistRemovals);
|
||||
blacklistAdditions, blacklistRemovals, increaseRequests,
|
||||
decreaseRequests);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -110,4 +110,9 @@ public class DefaultResourceCalculator extends ResourceCalculator {
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean fitsIn(Resource cluster,
|
||||
Resource smaller, Resource bigger) {
|
||||
return smaller.getMemory() <= bigger.getMemory();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -209,4 +209,10 @@ public class DominantResourceCalculator extends ResourceCalculator {
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean fitsIn(Resource cluster,
|
||||
Resource smaller, Resource bigger) {
|
||||
return smaller.getMemory() <= bigger.getMemory()
|
||||
&& smaller.getVirtualCores() <= bigger.getVirtualCores();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -171,4 +171,9 @@ public abstract class ResourceCalculator {
|
|||
*/
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -268,6 +268,11 @@ public class Resources {
|
|||
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()),
|
||||
Math.min(lhs.getVirtualCores(), rhs.getVirtualCores()));
|
||||
|
|
|
@ -42,6 +42,35 @@ public class TestResourceCalculator {
|
|||
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() {
|
||||
Resource clusterResource = Resource.newInstance(0, 0);
|
||||
|
@ -92,7 +121,6 @@ public class TestResourceCalculator {
|
|||
|
||||
}
|
||||
|
||||
|
||||
private void assertResourcesOperations(Resource clusterResource,
|
||||
Resource lhs, Resource rhs, boolean lessThan, boolean lessThanOrEqual,
|
||||
boolean greaterThan, boolean greaterThanOrEqual, Resource max,
|
||||
|
|
|
@ -19,12 +19,13 @@
|
|||
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.ContainerId;
|
||||
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;
|
||||
|
||||
|
@ -73,5 +74,5 @@ public interface NodeHeartbeatResponse {
|
|||
void setAreNodeLabelsAcceptedByRM(boolean areNodeLabelsAcceptedByRM);
|
||||
|
||||
List<Container> getContainersToDecrease();
|
||||
void addAllContainersToDecrease(List<Container> containersToDecrease);
|
||||
void addAllContainersToDecrease(Collection<Container> containersToDecrease);
|
||||
}
|
||||
|
|
|
@ -20,14 +20,15 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
|
|||
|
||||
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.ContainerId;
|
||||
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;
|
||||
|
@ -437,7 +438,7 @@ public class NodeHeartbeatResponsePBImpl extends
|
|||
|
||||
@Override
|
||||
public void addAllContainersToDecrease(
|
||||
final List<Container> containersToDecrease) {
|
||||
final Collection<Container> containersToDecrease) {
|
||||
if (containersToDecrease == null) {
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -452,10 +452,12 @@ public class ApplicationMasterService extends AbstractService implements
|
|||
}
|
||||
}
|
||||
|
||||
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 class ApplicationMasterService extends AbstractService implements
|
|||
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 class ApplicationMasterService extends AbstractService implements
|
|||
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()) {
|
||||
|
@ -541,6 +553,10 @@ public class ApplicationMasterService extends AbstractService implements
|
|||
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());
|
||||
|
||||
// add preemption to the allocateResponse message (if any)
|
||||
|
|
|
@ -56,6 +56,8 @@ public class RMAuditLogger {
|
|||
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";
|
||||
|
|
|
@ -22,8 +22,10 @@ import java.io.IOException;
|
|||
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.security.authorize.ProxyUsers;
|
|||
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.exceptions.YarnRuntimeException;
|
|||
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;
|
||||
|
||||
/**
|
||||
|
@ -108,6 +115,89 @@ public class RMServerUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalize container increase/decrease request, it will normalize and update
|
||||
* ContainerResourceChangeRequest.targetResource
|
||||
*
|
||||
* <pre>
|
||||
* - Throw exception when any other error happens
|
||||
* </pre>
|
||||
*/
|
||||
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 <code>InvalidResourceBlacklistRequestException </code> if the
|
||||
* resource is not able to be added to the blacklist.
|
||||
|
@ -124,6 +214,80 @@ public class RMServerUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if we have:
|
||||
* - Request for same containerId and different target resource
|
||||
* - If targetResources violates maximum/minimumAllocation
|
||||
*/
|
||||
public static void increaseDecreaseRequestSanityCheck(RMContext rmContext,
|
||||
List<ContainerResourceChangeRequest> incRequests,
|
||||
List<ContainerResourceChangeRequest> decRequests,
|
||||
Resource maximumAllocation) throws InvalidResourceRequestException {
|
||||
checkDuplicatedIncreaseDecreaseRequest(incRequests, decRequests);
|
||||
validateIncreaseDecreaseRequest(rmContext, incRequests, maximumAllocation,
|
||||
true);
|
||||
validateIncreaseDecreaseRequest(rmContext, decRequests, maximumAllocation,
|
||||
false);
|
||||
}
|
||||
|
||||
private static void checkDuplicatedIncreaseDecreaseRequest(
|
||||
List<ContainerResourceChangeRequest> incRequests,
|
||||
List<ContainerResourceChangeRequest> decRequests)
|
||||
throws InvalidResourceRequestException {
|
||||
String msg = "There're multiple increase or decrease container requests "
|
||||
+ "for same containerId=";
|
||||
Set<ContainerId> existedContainerIds = new HashSet<ContainerId>();
|
||||
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<ContainerResourceChangeRequest> 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
|
||||
* application attempt id. If not then it will throw
|
||||
|
|
|
@ -452,6 +452,8 @@ public class ResourceTrackerService extends AbstractService implements
|
|||
getResponseId() + 1, NodeAction.NORMAL, null, null, null, null,
|
||||
nextHeartBeatInterval);
|
||||
rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
|
||||
rmNode.updateNodeHeartbeatResponseForContainersDecreasing(
|
||||
nodeHeartBeatResponse);
|
||||
|
||||
populateKeys(request, nodeHeartBeatResponse);
|
||||
|
||||
|
@ -464,8 +466,9 @@ public class ResourceTrackerService extends AbstractService implements
|
|||
// 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
|
||||
|
|
|
@ -971,7 +971,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|||
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 class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
|||
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.
|
||||
|
|
|
@ -82,4 +82,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
|
|||
String getNodeHttpAddress();
|
||||
|
||||
String getNodeLabelExpression();
|
||||
|
||||
boolean hasIncreaseReservation();
|
||||
|
||||
void cancelIncreaseReservation();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -26,6 +26,10 @@ public enum RMContainerEventType {
|
|||
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 @@ public enum RMContainerEventType {
|
|||
// Source: ContainerAllocationExpirer
|
||||
EXPIRE,
|
||||
|
||||
RECOVER
|
||||
RECOVER,
|
||||
|
||||
// Source: Scheduler
|
||||
// Resource change approved by scheduler
|
||||
CHANGE_RESOURCE,
|
||||
|
||||
// NM reported resource change is done
|
||||
NM_DONE_CHANGE_RESOURCE
|
||||
}
|
||||
|
|
|
@ -118,7 +118,18 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
.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,
|
||||
|
@ -142,8 +153,6 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
// create the topology tables
|
||||
.installTopology();
|
||||
|
||||
|
||||
|
||||
private final StateMachine<RMContainerState, RMContainerEventType,
|
||||
RMContainerEvent> stateMachine;
|
||||
private final ReadLock readLock;
|
||||
|
@ -167,6 +176,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
private boolean isAMContainer;
|
||||
private List<ResourceRequest> resourceRequests;
|
||||
|
||||
private volatile boolean hasIncreaseReservation = false;
|
||||
|
||||
public RMContainerImpl(Container container,
|
||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||
RMContext rmContext) {
|
||||
|
@ -264,7 +275,12 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
|
||||
@Override
|
||||
public Resource getAllocatedResource() {
|
||||
return container.getResource();
|
||||
try {
|
||||
readLock.lock();
|
||||
return container.getResource();
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -471,8 +487,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
}
|
||||
}
|
||||
|
||||
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 class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -510,6 +532,70 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
}
|
||||
}
|
||||
|
||||
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 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
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 class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasIncreaseReservation() {
|
||||
return hasIncreaseReservation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancelIncreaseReservation() {
|
||||
hasIncreaseReservation = false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ 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;
|
||||
|
@ -146,4 +147,12 @@ public interface RMNode {
|
|||
* @return labels in this node
|
||||
*/
|
||||
public Set<String> getNodeLabels();
|
||||
|
||||
/**
|
||||
* Update containers to be decreased
|
||||
*/
|
||||
public void updateNodeHeartbeatResponseForContainersDecreasing(
|
||||
NodeHeartbeatResponse response);
|
||||
|
||||
public List<Container> pullNewlyIncreasedContainers();
|
||||
}
|
||||
|
|
|
@ -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<Container> toBeDecreasedContainers;
|
||||
|
||||
public RMNodeDecreaseContainerEvent(NodeId nodeId,
|
||||
List<Container> toBeDecreasedContainers) {
|
||||
super(nodeId, RMNodeEventType.DECREASE_CONTAINER);
|
||||
|
||||
this.toBeDecreasedContainers = toBeDecreasedContainers;
|
||||
}
|
||||
|
||||
public List<Container> getToBeDecreasedContainers() {
|
||||
return toBeDecreasedContainers;
|
||||
}
|
||||
}
|
|
@ -42,6 +42,7 @@ public enum RMNodeEventType {
|
|||
// Source: Container
|
||||
CONTAINER_ALLOCATED,
|
||||
CLEANUP_CONTAINER,
|
||||
DECREASE_CONTAINER,
|
||||
|
||||
// Source: RMAppAttempt
|
||||
FINISHED_CONTAINERS_PULLED_BY_AM,
|
||||
|
|
|
@ -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.classification.InterfaceStability.Unstable;
|
|||
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;
|
||||
|
@ -132,6 +137,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
private final List<ApplicationId> runningApplications =
|
||||
new ArrayList<ApplicationId>();
|
||||
|
||||
private final Map<ContainerId, Container> toBeDecreasedContainers =
|
||||
new HashMap<>();
|
||||
|
||||
private final Map<ContainerId, Container> nmReportedIncreasedContainers =
|
||||
new HashMap<>();
|
||||
|
||||
private NodeHeartbeatResponse latestNodeHeartBeatResponse = recordFactory
|
||||
.newRecordInstance(NodeHeartbeatResponse.class);
|
||||
|
||||
|
@ -180,6 +191,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
RMNodeEventType.RECONNECTED, new ReconnectNodeTransition())
|
||||
.addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
||||
RMNodeEventType.RESOURCE_UPDATE, new UpdateNodeResourceWhenRunningTransition())
|
||||
.addTransition(NodeState.RUNNING, NodeState.RUNNING,
|
||||
RMNodeEventType.DECREASE_CONTAINER,
|
||||
new DecreaseContainersTransition())
|
||||
.addTransition(NodeState.RUNNING, NodeState.SHUTDOWN,
|
||||
RMNodeEventType.SHUTDOWN,
|
||||
new DeactivateNodeTransition(NodeState.SHUTDOWN))
|
||||
|
@ -485,6 +499,24 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
}
|
||||
};
|
||||
|
||||
@VisibleForTesting
|
||||
public Collection<Container> 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() {
|
||||
|
||||
|
@ -837,6 +869,19 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
}
|
||||
}
|
||||
|
||||
public static class DecreaseContainersTransition
|
||||
implements SingleArcTransition<RMNodeImpl, RMNodeEvent> {
|
||||
|
||||
@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<RMNodeImpl, RMNodeEvent> {
|
||||
|
||||
|
@ -986,6 +1031,8 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
}
|
||||
|
||||
rmNode.handleContainerStatus(statusEvent.getContainers());
|
||||
rmNode.handleReportedIncreasedContainers(
|
||||
statusEvent.getNMReportedIncreasedContainers());
|
||||
|
||||
List<LogAggregationReport> logAggregationReportsForApps =
|
||||
statusEvent.getLogAggregationReportsForApps();
|
||||
|
@ -1080,6 +1127,34 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
return nlm.getLabelsOnNode(nodeId);
|
||||
}
|
||||
|
||||
private void handleReportedIncreasedContainers(
|
||||
List<Container> 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<ContainerStatus> containerStatuses) {
|
||||
// Filter the map to only obtain just launched containers and finished
|
||||
// containers.
|
||||
|
@ -1149,4 +1224,22 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Container> pullNewlyIncreasedContainers() {
|
||||
try {
|
||||
writeLock.lock();
|
||||
|
||||
if (nmReportedIncreasedContainers.isEmpty()) {
|
||||
return Collections.EMPTY_LIST;
|
||||
} else {
|
||||
List<Container> container =
|
||||
new ArrayList<Container>(nmReportedIncreasedContainers.values());
|
||||
nmReportedIncreasedContainers.clear();
|
||||
return container;
|
||||
}
|
||||
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 @@ public class RMNodeStatusEvent extends RMNodeEvent {
|
|||
private final NodeHeartbeatResponse latestResponse;
|
||||
private final List<ApplicationId> keepAliveAppIds;
|
||||
private List<LogAggregationReport> logAggregationReportsForApps;
|
||||
private final List<Container> nmReportedIncreasedContainers;
|
||||
|
||||
// Used by tests
|
||||
public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
|
||||
List<ContainerStatus> collection, List<ApplicationId> 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<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
|
||||
NodeHeartbeatResponse latestResponse,
|
||||
List<LogAggregationReport> logAggregationReportsForApps) {
|
||||
List<Container> nmReportedIncreasedContainers) {
|
||||
this(nodeId, nodeHealthStatus, collection, keepAliveAppIds, latestResponse,
|
||||
null, nmReportedIncreasedContainers);
|
||||
}
|
||||
|
||||
public RMNodeStatusEvent(NodeId nodeId, NodeHealthStatus nodeHealthStatus,
|
||||
List<ContainerStatus> collection, List<ApplicationId> keepAliveAppIds,
|
||||
NodeHeartbeatResponse latestResponse,
|
||||
List<LogAggregationReport> logAggregationReportsForApps,
|
||||
List<Container> 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 class RMNodeStatusEvent extends RMNodeEvent {
|
|||
List<LogAggregationReport> logAggregationReportsForApps) {
|
||||
this.logAggregationReportsForApps = logAggregationReportsForApps;
|
||||
}
|
||||
|
||||
public List<Container> getNMReportedIncreasedContainers() {
|
||||
return nmReportedIncreasedContainers == null ? Collections.EMPTY_LIST
|
||||
: nmReportedIncreasedContainers;
|
||||
}
|
||||
}
|
|
@ -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.ApplicationId;
|
|||
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.api.protocolrecords.NMContainerStatus;
|
|||
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.RMAppEventType;
|
|||
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 @@ public abstract class AbstractYarnScheduler
|
|||
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;
|
||||
|
@ -232,6 +245,55 @@ public abstract class AbstractYarnScheduler
|
|||
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<T> app =
|
||||
applications.get(applicationAttemptId.getApplicationId());
|
||||
|
@ -512,6 +574,36 @@ public abstract class AbstractYarnScheduler
|
|||
}
|
||||
}
|
||||
|
||||
protected void decreaseContainers(
|
||||
List<SchedContainerChangeRequest> 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);
|
||||
}
|
||||
|
@ -735,4 +827,56 @@ public abstract class AbstractYarnScheduler
|
|||
LOG.info("Updated the cluste max priority to maxClusterLevelAppPriority = "
|
||||
+ maxClusterLevelAppPriority);
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalize container increase/decrease request, and return
|
||||
* SchedulerContainerResourceChangeRequest according to given
|
||||
* ContainerResourceChangeRequest.
|
||||
*
|
||||
* <pre>
|
||||
* - Returns non-null value means validation succeeded
|
||||
* - Throw exception when any other error happens
|
||||
* </pre>
|
||||
*/
|
||||
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<SchedContainerChangeRequest>
|
||||
checkAndNormalizeContainerChangeRequests(
|
||||
List<ContainerResourceChangeRequest> changeRequests,
|
||||
boolean increase) {
|
||||
if (null == changeRequests || changeRequests.isEmpty()) {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
|
||||
List<SchedContainerChangeRequest> schedulerChangeRequests =
|
||||
new ArrayList<SchedContainerChangeRequest>();
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,9 @@ public class Allocation {
|
|||
final Set<ContainerId> fungibleContainers;
|
||||
final List<ResourceRequest> fungibleResources;
|
||||
final List<NMToken> nmTokens;
|
||||
final List<Container> increasedContainers;
|
||||
final List<Container> decreasedContainers;
|
||||
|
||||
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
|
@ -46,12 +49,22 @@ public class Allocation {
|
|||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
|
||||
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
||||
fungibleResources, nmTokens, null, null);
|
||||
}
|
||||
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
|
||||
List<Container> increasedContainers, List<Container> 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<Container> getContainers() {
|
||||
|
@ -78,4 +91,11 @@ public class Allocation {
|
|||
return nmTokens;
|
||||
}
|
||||
|
||||
public List<Container> getIncreasedContainers() {
|
||||
return increasedContainers;
|
||||
}
|
||||
|
||||
public List<Container> getDecreasedContainers() {
|
||||
return decreasedContainers;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,10 +20,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
|||
|
||||
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.classification.InterfaceStability.Unstable;
|
|||
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 @@ public class AppSchedulingInfo {
|
|||
|
||||
final Set<Priority> priorities = new TreeSet<Priority>(
|
||||
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
|
||||
final Map<Priority, Map<String, ResourceRequest>> requests =
|
||||
new ConcurrentHashMap<Priority, Map<String, ResourceRequest>>();
|
||||
final Map<Priority, Map<String, ResourceRequest>> resourceRequestMap =
|
||||
new ConcurrentHashMap<Priority, Map<String, ResourceRequest>>();
|
||||
final Map<NodeId, Map<Priority, Map<ContainerId,
|
||||
SchedContainerChangeRequest>>> increaseRequestMap =
|
||||
new ConcurrentHashMap<>();
|
||||
private Set<String> userBlacklist = new HashSet<>();
|
||||
private Set<String> amBlacklist = new HashSet<>();
|
||||
|
||||
|
@ -114,7 +121,7 @@ public class AppSchedulingInfo {
|
|||
*/
|
||||
private synchronized void clearRequests() {
|
||||
priorities.clear();
|
||||
requests.clear();
|
||||
resourceRequestMap.clear();
|
||||
LOG.info("Application " + applicationId + " requests cleared");
|
||||
}
|
||||
|
||||
|
@ -122,6 +129,170 @@ public class AppSchedulingInfo {
|
|||
return this.containerIdCounter.incrementAndGet();
|
||||
}
|
||||
|
||||
public boolean hasIncreaseRequest(NodeId nodeId) {
|
||||
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
|
||||
increaseRequestMap.get(nodeId);
|
||||
if (null == requestsOnNode) {
|
||||
return false;
|
||||
}
|
||||
return requestsOnNode.size() > 0;
|
||||
}
|
||||
|
||||
public Map<ContainerId, SchedContainerChangeRequest>
|
||||
getIncreaseRequests(NodeId nodeId, Priority priority) {
|
||||
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
|
||||
increaseRequestMap.get(nodeId);
|
||||
if (null == requestsOnNode) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return requestsOnNode.get(priority);
|
||||
}
|
||||
|
||||
public synchronized boolean updateIncreaseRequests(
|
||||
List<SchedContainerChangeRequest> increaseRequests) {
|
||||
boolean resourceUpdated = false;
|
||||
|
||||
for (SchedContainerChangeRequest r : increaseRequests) {
|
||||
NodeId nodeId = r.getRMContainer().getAllocatedNode();
|
||||
|
||||
Map<Priority, Map<ContainerId, SchedContainerChangeRequest>> 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<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
|
||||
increaseRequestMap.get(nodeId);
|
||||
if (null == requestsOnNode) {
|
||||
requestsOnNode =
|
||||
new HashMap<Priority, Map<ContainerId, SchedContainerChangeRequest>>();
|
||||
increaseRequestMap.put(nodeId, requestsOnNode);
|
||||
}
|
||||
|
||||
Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
|
||||
requestsOnNode.get(priority);
|
||||
if (null == requestsOnNodeWithPriority) {
|
||||
requestsOnNodeWithPriority =
|
||||
new TreeMap<ContainerId, SchedContainerChangeRequest>();
|
||||
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<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
|
||||
increaseRequestMap.get(nodeId);
|
||||
if (null == requestsOnNode) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Map<ContainerId, SchedContainerChangeRequest> 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<Priority, Map<ContainerId, SchedContainerChangeRequest>> requestsOnNode =
|
||||
increaseRequestMap.get(nodeId);
|
||||
if (null == requestsOnNode) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Map<ContainerId, SchedContainerChangeRequest> requestsOnNodeWithPriority =
|
||||
requestsOnNode.get(priority);
|
||||
if (null == requestsOnNodeWithPriority) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return requestsOnNodeWithPriority.get(containerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* The ApplicationMaster is updating resource requirements for the
|
||||
* application, by asking for more resources and releasing resources acquired
|
||||
|
@ -163,11 +334,11 @@ public class AppSchedulingInfo {
|
|||
}
|
||||
}
|
||||
|
||||
Map<String, ResourceRequest> asks = this.requests.get(priority);
|
||||
Map<String, ResourceRequest> asks = this.resourceRequestMap.get(priority);
|
||||
|
||||
if (asks == null) {
|
||||
asks = new ConcurrentHashMap<String, ResourceRequest>();
|
||||
this.requests.put(priority, asks);
|
||||
this.resourceRequestMap.put(priority, asks);
|
||||
this.priorities.add(priority);
|
||||
}
|
||||
lastRequest = asks.get(resourceName);
|
||||
|
@ -260,12 +431,12 @@ public class AppSchedulingInfo {
|
|||
|
||||
synchronized public Map<String, ResourceRequest> getResourceRequests(
|
||||
Priority priority) {
|
||||
return requests.get(priority);
|
||||
return resourceRequestMap.get(priority);
|
||||
}
|
||||
|
||||
public List<ResourceRequest> getAllResourceRequests() {
|
||||
List<ResourceRequest> ret = new ArrayList<ResourceRequest>();
|
||||
for (Map<String, ResourceRequest> r : requests.values()) {
|
||||
for (Map<String, ResourceRequest> r : resourceRequestMap.values()) {
|
||||
ret.addAll(r.values());
|
||||
}
|
||||
return ret;
|
||||
|
@ -273,7 +444,7 @@ public class AppSchedulingInfo {
|
|||
|
||||
synchronized public ResourceRequest getResourceRequest(Priority priority,
|
||||
String resourceName) {
|
||||
Map<String, ResourceRequest> nodeRequests = requests.get(priority);
|
||||
Map<String, ResourceRequest> nodeRequests = resourceRequestMap.get(priority);
|
||||
return (nodeRequests == null) ? null : nodeRequests.get(resourceName);
|
||||
}
|
||||
|
||||
|
@ -301,6 +472,50 @@ public class AppSchedulingInfo {
|
|||
}
|
||||
}
|
||||
|
||||
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 @@ public class AppSchedulingInfo {
|
|||
// 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 @@ public class AppSchedulingInfo {
|
|||
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 @@ public class AppSchedulingInfo {
|
|||
// 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 @@ public class AppSchedulingInfo {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
// also we need to check increase request
|
||||
if (!deactivate) {
|
||||
deactivate = increaseRequestMap.isEmpty();
|
||||
}
|
||||
|
||||
if (deactivate) {
|
||||
activeUsersManager.deactivateApplication(user, applicationId);
|
||||
}
|
||||
|
@ -457,7 +678,7 @@ public class AppSchedulingInfo {
|
|||
synchronized public void move(Queue newQueue) {
|
||||
QueueMetrics oldMetrics = queue.getMetrics();
|
||||
QueueMetrics newMetrics = newQueue.getMetrics();
|
||||
for (Map<String, ResourceRequest> asks : requests.values()) {
|
||||
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
||||
ResourceRequest request = asks.get(ResourceRequest.ANY);
|
||||
if (request != null) {
|
||||
oldMetrics.decrPendingResources(user, request.getNumContainers(),
|
||||
|
@ -484,7 +705,7 @@ public class AppSchedulingInfo {
|
|||
synchronized public void stop(RMAppAttemptState rmAppAttemptFinalState) {
|
||||
// clear pending resources metrics for the application
|
||||
QueueMetrics metrics = queue.getMetrics();
|
||||
for (Map<String, ResourceRequest> asks : requests.values()) {
|
||||
for (Map<String, ResourceRequest> asks : resourceRequestMap.values()) {
|
||||
ResourceRequest request = asks.get(ResourceRequest.ANY);
|
||||
if (request != null) {
|
||||
metrics.decrPendingResources(user, request.getNumContainers(),
|
||||
|
|
|
@ -373,17 +373,20 @@ public class QueueMetrics implements MetricsSource {
|
|||
}
|
||||
|
||||
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 class QueueMetrics implements MetricsSource {
|
|||
}
|
||||
|
||||
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);
|
||||
|
|
|
@ -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<SchedContainerChangeRequest> {
|
||||
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 "<container=" + getContainerId() + ", targetCapacity="
|
||||
+ targetCapacity + ", delta=" + deltaCapacity + ", node="
|
||||
+ getNodeId().toString() + ">";
|
||||
}
|
||||
}
|
|
@ -28,7 +28,7 @@ public class SchedulerApplication<T extends SchedulerApplicationAttempt> {
|
|||
|
||||
private Queue queue;
|
||||
private final String user;
|
||||
private T currentAttempt;
|
||||
private volatile T currentAttempt;
|
||||
private volatile Priority priority;
|
||||
|
||||
public SchedulerApplication(Queue queue, String user) {
|
||||
|
|
|
@ -19,11 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
|||
|
||||
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.AggregateAppR
|
|||
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 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
private AtomicLong firstAllocationRequestSentTime = new AtomicLong(0);
|
||||
private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
|
||||
|
||||
protected List<RMContainer> newlyAllocatedContainers =
|
||||
new ArrayList<RMContainer>();
|
||||
protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
|
||||
protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
|
||||
protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
|
||||
protected Set<NMToken> 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 class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
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 class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
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<NodeId, RMContainer> reservedContainers =
|
||||
this.reservedContainers.get(priority);
|
||||
|
@ -356,9 +368,31 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
+ " 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;
|
||||
}
|
||||
|
||||
|
@ -438,68 +472,99 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
return attemptResourceUsage.getUsed();
|
||||
}
|
||||
|
||||
public static class ContainersAndNMTokensAllocation {
|
||||
List<Container> containerList;
|
||||
List<NMToken> nmTokenList;
|
||||
|
||||
public ContainersAndNMTokensAllocation(List<Container> containerList,
|
||||
List<NMToken> 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;
|
||||
}
|
||||
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<Container> getContainerList() {
|
||||
return containerList;
|
||||
}
|
||||
|
||||
public List<NMToken> 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<Container> pullNewlyAllocatedContainers() {
|
||||
List<Container> returnContainerList =
|
||||
new ArrayList<Container>(newlyAllocatedContainers.size());
|
||||
List<NMToken> nmTokens = new ArrayList<NMToken>();
|
||||
for (Iterator<RMContainer> 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;
|
||||
}
|
||||
returnContainerList.add(container);
|
||||
i.remove();
|
||||
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
|
||||
RMContainerEventType.ACQUIRED));
|
||||
}
|
||||
return new ContainersAndNMTokensAllocation(returnContainerList, nmTokens);
|
||||
return returnContainerList;
|
||||
}
|
||||
|
||||
private synchronized List<Container> pullNewlyUpdatedContainers(
|
||||
Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
|
||||
List<Container> returnContainerList =
|
||||
new ArrayList<Container>(updatedContainerMap.size());
|
||||
for (Iterator<Entry<ContainerId, RMContainer>> 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();
|
||||
}
|
||||
}
|
||||
return returnContainerList;
|
||||
}
|
||||
|
||||
public synchronized List<Container> pullNewlyIncreasedContainers() {
|
||||
return pullNewlyUpdatedContainers(newlyIncreasedContainers, true);
|
||||
}
|
||||
|
||||
public synchronized List<Container> pullNewlyDecreasedContainers() {
|
||||
return pullNewlyUpdatedContainers(newlyDecreasedContainers, false);
|
||||
}
|
||||
|
||||
public synchronized List<NMToken> pullUpdatedNMTokens() {
|
||||
List<NMToken> returnList = new ArrayList<NMToken>(updatedNMTokens);
|
||||
updatedNMTokens.clear();
|
||||
return returnList;
|
||||
}
|
||||
|
||||
public boolean isWaitingForAMContainer(ApplicationId applicationId) {
|
||||
|
@ -770,4 +835,50 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
return attemptResourceUsage;
|
||||
}
|
||||
|
||||
public synchronized boolean removeIncreaseRequest(NodeId nodeId,
|
||||
Priority priority, ContainerId containerId) {
|
||||
return appSchedulingInfo.removeIncreaseRequest(nodeId, priority,
|
||||
containerId);
|
||||
}
|
||||
|
||||
public synchronized boolean updateIncreaseRequests(
|
||||
List<SchedContainerChangeRequest> 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);
|
||||
}
|
||||
}
|
|
@ -158,6 +158,37 @@ public abstract class SchedulerNode {
|
|||
+ " 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.
|
||||
*
|
||||
|
|
|
@ -361,7 +361,7 @@ public class SchedulerUtils {
|
|||
}
|
||||
|
||||
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.
|
||||
|
@ -372,11 +372,10 @@ public class SchedulerUtils {
|
|||
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,
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||
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 interface YarnScheduler extends EventHandler<SchedulerEvent> {
|
|||
* @param release
|
||||
* @param blacklistAdditions
|
||||
* @param blacklistRemovals
|
||||
* @param increaseRequests
|
||||
* @param decreaseRequests
|
||||
* @return the {@link Allocation} for the application
|
||||
*/
|
||||
@Public
|
||||
@Stable
|
||||
Allocation
|
||||
allocate(ApplicationAttemptId appAttemptId,
|
||||
List<ResourceRequest> ask,
|
||||
List<ContainerId> release,
|
||||
List<String> blacklistAdditions,
|
||||
List<String> blacklistRemovals);
|
||||
Allocation allocate(ApplicationAttemptId appAttemptId,
|
||||
List<ResourceRequest> ask, List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests);
|
||||
|
||||
/**
|
||||
* Get node resource usage report.
|
||||
|
|
|
@ -43,10 +43,10 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity;
|
|||
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 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
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.
|
||||
|
@ -341,21 +341,26 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
}
|
||||
|
||||
synchronized void allocateResource(Resource clusterResource,
|
||||
Resource resource, String nodePartition) {
|
||||
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 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
}
|
||||
|
||||
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.
|
||||
|
|
|
@ -41,6 +41,7 @@ public class CSAssignment {
|
|||
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 class CSAssignment {
|
|||
public AssignmentInformation getAssignmentInformation() {
|
||||
return this.assignmentInformation;
|
||||
}
|
||||
|
||||
public boolean isIncreasedAllocation() {
|
||||
return increaseAllocation;
|
||||
}
|
||||
|
||||
public void setIncreasedAllocation(boolean flag) {
|
||||
increaseAllocation = flag;
|
||||
}
|
||||
}
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
|
|||
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;
|
||||
|
||||
|
@ -218,6 +219,14 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
|
|||
RMContainerEventType event, CSQueue childQueue,
|
||||
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 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
|
|||
* 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);
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
|||
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.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.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.RMContainer;
|
|||
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.QueueMetrics;
|
|||
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 @@ public class CapacityScheduler extends
|
|||
}
|
||||
|
||||
@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<ResourceRequest> ask, List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
|
||||
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||
if (application == null) {
|
||||
|
@ -855,6 +864,14 @@ public class CapacityScheduler extends
|
|||
ask, getResourceCalculator(), getClusterResource(),
|
||||
getMinimumResourceCapability(), getMaximumResourceCapability());
|
||||
|
||||
// Pre-process increase requests
|
||||
List<SchedContainerChangeRequest> normalizedIncreaseRequests =
|
||||
checkAndNormalizeContainerChangeRequests(increaseRequests, true);
|
||||
|
||||
// Pre-process decrease requests
|
||||
List<SchedContainerChangeRequest> normalizedDecreaseRequests =
|
||||
checkAndNormalizeContainerChangeRequests(decreaseRequests, false);
|
||||
|
||||
// Release containers
|
||||
releaseContainers(release, application);
|
||||
|
||||
|
@ -870,8 +887,8 @@ public class CapacityScheduler extends
|
|||
return EMPTY_ALLOCATION;
|
||||
}
|
||||
|
||||
// Process resource requests
|
||||
if (!ask.isEmpty()) {
|
||||
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: pre-update " + applicationAttemptId +
|
||||
" ask size =" + ask.size());
|
||||
|
@ -889,6 +906,12 @@ public class CapacityScheduler extends
|
|||
}
|
||||
}
|
||||
|
||||
// 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
|
||||
application.updateAMBlacklist(
|
||||
|
@ -897,6 +920,9 @@ public class CapacityScheduler extends
|
|||
application.updateBlacklist(blacklistAdditions, blacklistRemovals);
|
||||
}
|
||||
|
||||
// Decrease containers
|
||||
decreaseContainers(normalizedDecreaseRequests, application);
|
||||
|
||||
allocation = application.getAllocation(getResourceCalculator(),
|
||||
clusterResource, getMinimumResourceCapability());
|
||||
}
|
||||
|
@ -958,6 +984,13 @@ public class CapacityScheduler extends
|
|||
containerLaunchedOnNode(launchedContainer.getContainerId(), node);
|
||||
}
|
||||
|
||||
// Processing the newly increased containers
|
||||
List<Container> newlyIncreasedContainers =
|
||||
nm.pullNewlyIncreasedContainers();
|
||||
for (Container container : newlyIncreasedContainers) {
|
||||
containerIncreasedOnNode(container.getId(), node, container);
|
||||
}
|
||||
|
||||
// Process completed containers
|
||||
int releasedContainers = 0;
|
||||
for (ContainerStatus completedContainer : completedContainers) {
|
||||
|
@ -1443,6 +1476,50 @@ public class CapacityScheduler extends
|
|||
}
|
||||
}
|
||||
|
||||
@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
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
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.ActiveUsersManage
|
|||
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 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
}
|
||||
|
||||
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 class LeafQueue extends AbstractCSQueue {
|
|||
CSAssignment assignment =
|
||||
application.assignContainers(clusterResource, node,
|
||||
currentResourceLimits, schedulingMode, reservedContainer);
|
||||
handleExcessReservedContainer(clusterResource, assignment);
|
||||
handleExcessReservedContainer(clusterResource, assignment, node,
|
||||
application);
|
||||
return assignment;
|
||||
}
|
||||
}
|
||||
|
@ -824,7 +829,8 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
// 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 class LeafQueue extends AbstractCSQueue {
|
|||
// 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;
|
||||
|
@ -1087,12 +1094,52 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
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,
|
||||
FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer,
|
||||
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 class LeafQueue extends AbstractCSQueue {
|
|||
orderingPolicy.containerReleased(application, rmContainer);
|
||||
|
||||
releaseResource(clusterResource, application, container.getResource(),
|
||||
node.getPartition(), rmContainer);
|
||||
node.getPartition(), rmContainer, false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1137,8 +1184,10 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
|
||||
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 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
|
||||
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 class LeafQueue extends AbstractCSQueue {
|
|||
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 class LeafQueue extends AbstractCSQueue {
|
|||
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 class LeafQueue extends AbstractCSQueue {
|
|||
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()
|
||||
|
@ -1483,6 +1533,39 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
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<FiCaSchedulerApp>
|
||||
getPendingAppsOrderingPolicy() {
|
||||
return pendingOrderingPolicy;
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
|
|||
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 class ParentQueue extends AbstractCSQueue {
|
|||
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 class ParentQueue extends AbstractCSQueue {
|
|||
.addAll(
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getReservationDetails());
|
||||
assignment.setIncreasedAllocation(assignedToChild
|
||||
.isIncreasedAllocation());
|
||||
|
||||
LOG.info("assignedContainer" +
|
||||
" queue=" + getQueueName() +
|
||||
|
@ -616,6 +619,73 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
}
|
||||
}
|
||||
|
||||
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<CSQueue> 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 class ParentQueue extends AbstractCSQueue {
|
|||
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<CSQueue> 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 class ParentQueue extends AbstractCSQueue {
|
|||
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 class ParentQueue extends AbstractCSQueue {
|
|||
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 class ParentQueue extends AbstractCSQueue {
|
|||
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="
|
||||
|
|
|
@ -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:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Select request: Select a request to allocate. E.g. select a resource
|
||||
* request based on requirement/priority/locality.</li>
|
||||
* <li>Check if a given resource can be allocated based on resource
|
||||
* availability</li>
|
||||
* <li>Do allocation: this will decide/create allocated/reserved
|
||||
* container, this will also update metrics</li>
|
||||
* </ul>
|
||||
*/
|
||||
public abstract CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, RMContainer reservedContainer);
|
||||
}
|
|
@ -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.server.resourcemanager.scheduler.common.fica.FiCaS
|
|||
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);
|
||||
|
||||
FiCaSchedulerApp application;
|
||||
final ResourceCalculator rc;
|
||||
final RMContext rmContext;
|
||||
public class ContainerAllocator extends AbstractContainerAllocator {
|
||||
AbstractContainerAllocator increaseContainerAllocator;
|
||||
AbstractContainerAllocator regularContainerAllocator;
|
||||
|
||||
public ContainerAllocator(FiCaSchedulerApp application,
|
||||
ResourceCalculator rc, RMContext rmContext) {
|
||||
this.application = application;
|
||||
this.rc = rc;
|
||||
this.rmContext = rmContext;
|
||||
super(application, rc, rmContext);
|
||||
|
||||
increaseContainerAllocator =
|
||||
new IncreaseContainerAllocator(application, rc, rmContext);
|
||||
regularContainerAllocator =
|
||||
new RegularContainerAllocator(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);
|
||||
}
|
||||
|
||||
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:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Select request: Select a request to allocate. E.g. select a resource
|
||||
* request based on requirement/priority/locality.</li>
|
||||
* <li>Check if a given resource can be allocated based on resource
|
||||
* availability</li>
|
||||
* <li>Do allocation: this will decide/create allocated/reserved
|
||||
* container, this will also update metrics</li>
|
||||
* </ul>
|
||||
*/
|
||||
public abstract CSAssignment assignContainers(Resource clusterResource,
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, RMContainer reservedContainer);
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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<ContainerId, SchedContainerChangeRequest> 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<Entry<ContainerId, SchedContainerChangeRequest>> iter =
|
||||
increaseRequestMap.entrySet().iterator();
|
||||
List<SchedContainerChangeRequest> toBeRemovedRequests =
|
||||
new ArrayList<>();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
Entry<ContainerId, SchedContainerChangeRequest> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|||
* 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 class RegularContainerAllocator extends ContainerAllocator {
|
|||
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 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|||
// 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 @@ public class RegularContainerAllocator extends ContainerAllocator {
|
|||
}
|
||||
|
||||
// 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.
|
||||
|
|
|
@ -32,6 +32,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.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.Capacity
|
|||
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 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
|
||||
private ResourceScheduler scheduler;
|
||||
|
||||
private ContainerAllocator containerAllocator;
|
||||
private AbstractContainerAllocator containerAllocator;
|
||||
|
||||
public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId,
|
||||
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
||||
|
@ -118,7 +119,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
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 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
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<NodeId, RMContainer> reservedContainers =
|
||||
this.reservedContainers.get(priority);
|
||||
|
||||
|
@ -241,7 +244,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
// 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 class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
ResourceRequest rr = ResourceRequest.newInstance(
|
||||
Priority.UNDEFINED, ResourceRequest.ANY,
|
||||
minimumAllocation, numCont);
|
||||
ContainersAndNMTokensAllocation allocation =
|
||||
pullNewlyAllocatedContainersAndNMTokens();
|
||||
List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
|
||||
List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
|
||||
List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
|
||||
List<NMToken> 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 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
|
||||
for (Map.Entry<NodeId, RMContainer> 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;
|
||||
|
@ -375,6 +388,25 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
((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) {
|
||||
// Update reserved metrics if this is the first reservation
|
||||
|
|
|
@ -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.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.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.ActiveUsersManage
|
|||
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;
|
||||
|
@ -914,7 +923,9 @@ public class FairScheduler extends
|
|||
@Override
|
||||
public Allocation allocate(ApplicationAttemptId appAttemptId,
|
||||
List<ResourceRequest> ask, List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
|
||||
// Make sure this application exists
|
||||
FSAppAttempt application = getSchedulerApp(appAttemptId);
|
||||
|
@ -973,18 +984,17 @@ public class FairScheduler extends
|
|||
application.updateBlacklist(blacklistAdditions, blacklistRemovals);
|
||||
}
|
||||
|
||||
ContainersAndNMTokensAllocation allocation =
|
||||
application.pullNewlyAllocatedContainersAndNMTokens();
|
||||
|
||||
List<Container> 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());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1725,4 +1735,11 @@ public class FairScheduler extends
|
|||
}
|
||||
return targetQueueName;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decreaseContainer(
|
||||
SchedContainerChangeRequest decreaseRequest,
|
||||
SchedulerApplicationAttempt attempt) {
|
||||
// TODO Auto-generated method stub
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,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.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.QueueMetrics;
|
|||
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 class FifoScheduler extends
|
|||
}
|
||||
|
||||
@Override
|
||||
public Allocation allocate(
|
||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||
List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
|
||||
List<ResourceRequest> ask, List<ContainerId> release,
|
||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> decreaseRequests) {
|
||||
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||
if (application == null) {
|
||||
LOG.error("Calling allocate on removed " +
|
||||
|
@ -364,12 +367,10 @@ public class FifoScheduler extends
|
|||
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 class FifoScheduler extends
|
|||
public Resource getUsedResource() {
|
||||
return usedResource;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decreaseContainer(
|
||||
SchedContainerChangeRequest decreaseRequest,
|
||||
SchedulerApplicationAttempt attempt) {
|
||||
// TODO Auto-generated method stub
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -323,7 +323,7 @@ public class Application {
|
|||
// Get resources from the ResourceManager
|
||||
Allocation allocation = resourceManager.getResourceScheduler().allocate(
|
||||
applicationAttemptId, new ArrayList<ResourceRequest>(ask),
|
||||
new ArrayList<ContainerId>(), null, null);
|
||||
new ArrayList<ContainerId>(), null, null, null, null);
|
||||
System.out.println("-=======" + applicationAttemptId);
|
||||
System.out.println("----------" + resourceManager.getRMContext().getRMApps()
|
||||
.get(applicationId).getRMAppAttempt(applicationAttemptId));
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
|
|||
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;
|
||||
|
@ -236,6 +237,14 @@ public class MockAM {
|
|||
return allocate(req);
|
||||
}
|
||||
|
||||
public AllocateResponse sendContainerResizingRequest(
|
||||
List<ContainerResourceChangeRequest> increaseRequests,
|
||||
List<ContainerResourceChangeRequest> 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 {
|
||||
UserGroupInformation ugi =
|
||||
|
|
|
@ -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 class MockNodes {
|
|||
}
|
||||
return CommonNodeLabelsManager.EMPTY_STRING_SET;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateNodeHeartbeatResponseForContainersDecreasing(
|
||||
NodeHeartbeatResponse response) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Container> pullNewlyIncreasedContainers() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
};
|
||||
|
||||
private static RMNode buildRMNode(int rack, final Resource perNode,
|
||||
|
|
|
@ -191,6 +191,19 @@ public class MockRM extends ResourceManager {
|
|||
}
|
||||
}
|
||||
|
||||
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;
|
||||
|
|
|
@ -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 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.impl.pb.AllocateRequestPBImpl;
|
||||
import org.apache.hadoop.yarn.api.records.*;
|
||||
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.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.scheduler.ResourceScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.Assert;
|
||||
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 static java.lang.Thread.sleep;
|
||||
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.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.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;
|
||||
|
||||
public class TestApplicationMasterService {
|
||||
private static final Log LOG = LogFactory
|
||||
|
@ -344,6 +351,92 @@ public class TestApplicationMasterService {
|
|||
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 {
|
||||
|
||||
public MyResourceManager(YarnConfiguration conf) {
|
||||
|
@ -354,4 +447,15 @@ public class TestApplicationMasterService {
|
|||
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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|||
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 class TestAMRestart {
|
|||
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<Container>();
|
||||
|
@ -365,7 +370,7 @@ public class TestAMRestart {
|
|||
// 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<NMToken> transferredTokens = registerResponse.getNMTokensFromPreviousAttempts();
|
||||
|
@ -430,7 +435,7 @@ public class TestAMRestart {
|
|||
|
||||
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);
|
||||
|
|
|
@ -165,7 +165,7 @@ public class TestRMAppLogAggregationStatus {
|
|||
node1ReportForApp.add(report1);
|
||||
node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
|
||||
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
|
||||
null, node1ReportForApp));
|
||||
null, node1ReportForApp, null));
|
||||
|
||||
List<LogAggregationReport> node2ReportForApp =
|
||||
new ArrayList<LogAggregationReport>();
|
||||
|
@ -177,7 +177,7 @@ public class TestRMAppLogAggregationStatus {
|
|||
node2ReportForApp.add(report2);
|
||||
node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
|
||||
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), 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 class TestRMAppLogAggregationStatus {
|
|||
node1ReportForApp2.add(report1_2);
|
||||
node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
|
||||
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
|
||||
null, node1ReportForApp2));
|
||||
null, node1ReportForApp2, null));
|
||||
|
||||
// verify that the log aggregation status for node1
|
||||
// has been changed
|
||||
|
@ -284,7 +284,7 @@ public class TestRMAppLogAggregationStatus {
|
|||
// 10 diagnostic messages/failure messages
|
||||
node1.handle(new RMNodeStatusEvent(node1.getNodeID(), NodeHealthStatus
|
||||
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
|
||||
null, node1ReportForApp3));
|
||||
null, node1ReportForApp3, null));
|
||||
|
||||
logAggregationStatus = rmApp.getLogAggregationReportsForApp();
|
||||
Assert.assertEquals(2, logAggregationStatus.size());
|
||||
|
@ -329,7 +329,7 @@ public class TestRMAppLogAggregationStatus {
|
|||
node2ReportForApp2.add(report2_3);
|
||||
node2.handle(new RMNodeStatusEvent(node2.getNodeID(), NodeHealthStatus
|
||||
.newInstance(true, null, 0), new ArrayList<ContainerStatus>(), null,
|
||||
null, node2ReportForApp2));
|
||||
null, node2ReportForApp2, null));
|
||||
Assert.assertEquals(LogAggregationStatus.FAILED,
|
||||
rmApp.getLogAggregationStatusForAppReport());
|
||||
logAggregationStatus = rmApp.getLogAggregationReportsForApp();
|
||||
|
|
|
@ -465,10 +465,9 @@ public class TestRMAppAttemptTransitions {
|
|||
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 @@ public class TestRMAppAttemptTransitions {
|
|||
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 @@ public class TestRMAppAttemptTransitions {
|
|||
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())).
|
||||
|
@ -1511,10 +1504,9 @@ public class TestRMAppAttemptTransitions {
|
|||
@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<Allocation>() {
|
||||
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<Allocation>() {
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
|
|||
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;
|
||||
|
@ -192,6 +191,10 @@ public class TestRMContainerImpl {
|
|||
Container container = BuilderUtils.newContainer(containerId, nodeId,
|
||||
"host:3465", resource, priority, null);
|
||||
|
||||
ConcurrentMap<ApplicationId, RMApp> appMap = new ConcurrentHashMap<>();
|
||||
RMApp rmApp = mock(RMApp.class);
|
||||
appMap.putIfAbsent(appId, rmApp);
|
||||
|
||||
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
|
||||
SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
|
@ -200,6 +203,7 @@ public class TestRMContainerImpl {
|
|||
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,12 +239,119 @@ public class TestRMContainerImpl {
|
|||
rmContainer.handle(new RMContainerFinishedEvent(containerId,
|
||||
containerStatus, RMContainerEventType.EXPIRE));
|
||||
drainDispatcher.await();
|
||||
assertEquals(RMContainerState.RUNNING, rmContainer.getState());
|
||||
verify(writer, never()).containerFinished(any(RMContainer.class));
|
||||
verify(publisher, never()).containerFinished(any(RMContainer.class),
|
||||
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<RMAppAttemptEvent> 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<ApplicationId, RMApp> apps =
|
||||
new ConcurrentHashMap<ApplicationId, RMApp>();
|
||||
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());
|
||||
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 {
|
||||
Configuration conf = new Configuration();
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.security.PrivilegedAction;
|
|||
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.ApplicationResourceUsageReport;
|
|||
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.RMAppAttemptI
|
|||
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;
|
||||
|
@ -139,7 +141,6 @@ import org.junit.After;
|
|||
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;
|
||||
|
@ -678,11 +679,11 @@ public class TestCapacityScheduler {
|
|||
// Verify the blacklist can be updated independent of requesting containers
|
||||
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||
Collections.<ContainerId>emptyList(),
|
||||
Collections.singletonList(host), null);
|
||||
Collections.singletonList(host), null, null, null);
|
||||
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||
Collections.<ContainerId>emptyList(), null,
|
||||
Collections.singletonList(host));
|
||||
Collections.singletonList(host), null, null);
|
||||
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||
rm.stop();
|
||||
}
|
||||
|
@ -777,7 +778,7 @@ public class TestCapacityScheduler {
|
|||
cs.allocate(appAttemptId1,
|
||||
Collections.<ResourceRequest>singletonList(r1),
|
||||
Collections.<ContainerId>emptyList(),
|
||||
null, null);
|
||||
null, null, null, null);
|
||||
|
||||
//And this will result in container assignment for app1
|
||||
CapacityScheduler.schedule(cs);
|
||||
|
@ -794,7 +795,7 @@ public class TestCapacityScheduler {
|
|||
cs.allocate(appAttemptId2,
|
||||
Collections.<ResourceRequest>singletonList(r2),
|
||||
Collections.<ContainerId>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
|
||||
|
@ -2907,7 +2908,7 @@ public class TestCapacityScheduler {
|
|||
|
||||
Allocation allocate =
|
||||
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
||||
Collections.<ContainerId> emptyList(), null, null);
|
||||
Collections.<ContainerId> emptyList(), null, null, null, null);
|
||||
|
||||
Assert.assertNotNull(attempt);
|
||||
|
||||
|
@ -2923,7 +2924,7 @@ public class TestCapacityScheduler {
|
|||
|
||||
allocate =
|
||||
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
||||
Collections.<ContainerId> emptyList(), null, null);
|
||||
Collections.<ContainerId> emptyList(), null, null, null, null);
|
||||
|
||||
// All resources should be sent as headroom
|
||||
Assert.assertEquals(newResource, allocate.getResourceLimit());
|
||||
|
@ -3084,7 +3085,107 @@ public class TestCapacityScheduler {
|
|||
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)
|
||||
|
@ -3146,4 +3247,15 @@ public class TestCapacityScheduler {
|
|||
+ 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");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -132,11 +132,11 @@ public class TestChildQueueOrder {
|
|||
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
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|||
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.Assert;
|
|||
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 class TestContainerAllocation {
|
|||
|
||||
// acquire the container.
|
||||
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
||||
List<Container> containers =
|
||||
am1.allocate(new ArrayList<ResourceRequest>(),
|
||||
new ArrayList<ContainerId>()).getAllocatedContainers();
|
||||
// not able to fetch the container;
|
||||
Assert.assertEquals(0, containers.size());
|
||||
|
||||
SecurityUtilTestHelper.setTokenServiceUseIp(false);
|
||||
List<Container> containers;
|
||||
try {
|
||||
containers =
|
||||
am1.allocate(new ArrayList<ResourceRequest>(),
|
||||
new ArrayList<ContainerId>()).getAllocatedContainers();
|
||||
// not able to fetch the container;
|
||||
Assert.assertEquals(0, containers.size());
|
||||
} finally {
|
||||
SecurityUtilTestHelper.setTokenServiceUseIp(false);
|
||||
}
|
||||
containers =
|
||||
am1.allocate(new ArrayList<ResourceRequest>(),
|
||||
new ArrayList<ContainerId>()).getAllocatedContainers();
|
||||
|
@ -315,21 +314,24 @@ public class TestContainerAllocation {
|
|||
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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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<Container> 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<ContainerResourceChangeRequest> 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<ContainerResourceChangeRequest> 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<Container> 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<Container> 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();
|
||||
}
|
||||
}
|
|
@ -770,9 +770,9 @@ public class TestLeafQueue {
|
|||
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);
|
||||
|
|
|
@ -144,11 +144,11 @@ public class TestParentQueue {
|
|||
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
|
||||
|
|
|
@ -60,6 +60,9 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|||
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 class TestReservations {
|
|||
@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 class TestReservations {
|
|||
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 class TestReservations {
|
|||
|
||||
// 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);
|
||||
|
|
|
@ -169,7 +169,7 @@ public class FairSchedulerTestBase {
|
|||
resourceManager.getRMContext().getRMApps()
|
||||
.put(id.getApplicationId(), rmApp);
|
||||
|
||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
return id;
|
||||
}
|
||||
|
||||
|
@ -195,7 +195,7 @@ public class FairSchedulerTestBase {
|
|||
resourceManager.getRMContext().getRMApps()
|
||||
.put(id.getApplicationId(), rmApp);
|
||||
|
||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
return id;
|
||||
}
|
||||
|
||||
|
@ -217,7 +217,7 @@ public class FairSchedulerTestBase {
|
|||
ResourceRequest request, ApplicationAttemptId attId) {
|
||||
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
|
||||
ask.add(request);
|
||||
scheduler.allocate(attId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(attId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
}
|
||||
|
||||
protected void createApplicationWithAMResource(ApplicationAttemptId attId,
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
|
|||
List<ResourceRequest> ask = new ArrayList<>();
|
||||
ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
|
||||
scheduler.allocate(
|
||||
appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
||||
|
||||
// Advance time and let continuous scheduling kick in
|
||||
|
|
|
@ -1456,7 +1456,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
ResourceRequest request1 =
|
||||
createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
|
||||
ask1.add(request1);
|
||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
// Second ask, queue2 requests 1 large + (2 * minReqSize)
|
||||
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
||||
|
@ -1466,14 +1466,14 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
false);
|
||||
ask2.add(request2);
|
||||
ask2.add(request3);
|
||||
scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
// Third ask, queue2 requests 1 large
|
||||
List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
|
||||
ResourceRequest request4 =
|
||||
createResourceRequest(2 * minReqSize, ResourceRequest.ANY, 1, 1, true);
|
||||
ask3.add(request4);
|
||||
scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
scheduler.update();
|
||||
|
||||
|
@ -2795,7 +2795,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
// Complete container
|
||||
scheduler.allocate(attId, new ArrayList<ResourceRequest>(),
|
||||
Arrays.asList(containerId), null, null);
|
||||
Arrays.asList(containerId), null, null, null, null);
|
||||
assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB());
|
||||
assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
||||
|
||||
|
@ -2887,7 +2887,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
|
||||
|
||||
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
|
||||
null);
|
||||
null, null, null);
|
||||
|
||||
// node 1 checks in
|
||||
scheduler.update();
|
||||
|
@ -3283,7 +3283,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
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<ContainerId>(), null, null);
|
||||
scheduler.allocate(attId1, update, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
// then node2 should get the container
|
||||
scheduler.handle(node2UpdateEvent);
|
||||
|
@ -3330,7 +3330,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
|
||||
1, 1, false);
|
||||
scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest),
|
||||
new ArrayList<ContainerId>(), null, null);
|
||||
new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
scheduler.handle(nodeUpdateEvent);
|
||||
assertEquals(0, app.getReservedContainers().size());
|
||||
|
@ -4332,7 +4332,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
ResourceRequest request =
|
||||
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
|
||||
ask.add(request);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
// waiting for continuous_scheduler_sleep_time
|
||||
// at least one pass
|
||||
|
@ -4352,7 +4352,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
ask.clear();
|
||||
ask.add(request);
|
||||
scheduler.stop();
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
scheduler.continuousSchedulingAttempt();
|
||||
Assert.assertEquals(2048, app.getCurrentConsumption().getMemory());
|
||||
Assert.assertEquals(2, app.getCurrentConsumption().getVirtualCores());
|
||||
|
@ -4452,7 +4452,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
ask1.add(request1);
|
||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
|
||||
null);
|
||||
null, null, null);
|
||||
|
||||
String hostName = "127.0.0.1";
|
||||
RMNode node1 = MockNodes.newNodeInfo(1,
|
||||
|
@ -4584,7 +4584,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
List<Container> containers = scheduler.allocate(appAttemptId,
|
||||
Collections.<ResourceRequest> emptyList(),
|
||||
Collections.<ContainerId> emptyList(), null, null).getContainers();
|
||||
Collections.<ContainerId> emptyList(), null, null, null, null).getContainers();
|
||||
|
||||
// Now with updated ResourceRequest, a container is allocated for AM.
|
||||
Assert.assertTrue(containers.size() == 1);
|
||||
|
@ -4613,11 +4613,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
// Verify the blacklist can be updated independent of requesting containers
|
||||
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||
Collections.<ContainerId>emptyList(),
|
||||
Collections.singletonList(host), null);
|
||||
Collections.singletonList(host), null, null, null);
|
||||
assertTrue(app.isBlacklisted(host));
|
||||
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||
Collections.<ContainerId>emptyList(), null,
|
||||
Collections.singletonList(host));
|
||||
Collections.singletonList(host), null, null);
|
||||
assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host));
|
||||
|
||||
List<ResourceRequest> update = Arrays.asList(
|
||||
|
@ -4626,7 +4626,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
// Verify a container does not actually get placed on the blacklisted host
|
||||
scheduler.allocate(appAttemptId, update,
|
||||
Collections.<ContainerId>emptyList(),
|
||||
Collections.singletonList(host), null);
|
||||
Collections.singletonList(host), null, null, null);
|
||||
assertTrue(app.isBlacklisted(host));
|
||||
scheduler.update();
|
||||
scheduler.handle(updateEvent);
|
||||
|
@ -4636,7 +4636,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
// Verify a container gets placed on the empty blacklist
|
||||
scheduler.allocate(appAttemptId, update,
|
||||
Collections.<ContainerId>emptyList(), null,
|
||||
Collections.singletonList(host));
|
||||
Collections.singletonList(host), null, null);
|
||||
assertFalse(app.isBlacklisted(host));
|
||||
createSchedulingRequest(GB, "root.default", "user", 1);
|
||||
scheduler.update();
|
||||
|
|
|
@ -263,7 +263,7 @@ public class TestFifoScheduler {
|
|||
ask.add(nodeLocal);
|
||||
ask.add(rackLocal);
|
||||
ask.add(any);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
|
||||
|
||||
|
@ -365,7 +365,7 @@ public class TestFifoScheduler {
|
|||
ask.add(nodeLocal);
|
||||
ask.add(rackLocal);
|
||||
ask.add(any);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
|
||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
||||
|
||||
// Before the node update event, there are one local request
|
||||
Assert.assertEquals(1, nodeLocal.getNumContainers());
|
||||
|
@ -941,7 +941,7 @@ public class TestFifoScheduler {
|
|||
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 class TestFifoScheduler {
|
|||
// 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<Container> containerList = allocation2.getContainers();
|
||||
for (Container container : containerList) {
|
||||
|
@ -971,29 +971,29 @@ public class TestFifoScheduler {
|
|||
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 class TestFifoScheduler {
|
|||
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
|
||||
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<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
||||
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());
|
||||
|
||||
|
|
Loading…
Reference in New Issue