YARN-5959. RM changes to support change of container ExecutionType. (Arun Suresh via wangda)
This commit is contained in:
parent
a605ff36a5
commit
0a55bd841e
@ -110,7 +110,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
import org.apache.hadoop.yarn.event.DrainDispatcher;
|
import org.apache.hadoop.yarn.event.DrainDispatcher;
|
||||||
@ -132,6 +131,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||||
@ -1707,8 +1707,7 @@ public synchronized Allocation allocate(
|
|||||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||||
List<ContainerId> release, List<String> blacklistAdditions,
|
List<ContainerId> release, List<String> blacklistAdditions,
|
||||||
List<String> blacklistRemovals,
|
List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||||
for (ResourceRequest req : ask) {
|
for (ResourceRequest req : ask) {
|
||||||
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
||||||
@ -1723,7 +1722,7 @@ public synchronized Allocation allocate(
|
|||||||
lastBlacklistRemovals = blacklistRemovals;
|
lastBlacklistRemovals = blacklistRemovals;
|
||||||
Allocation allocation = super.allocate(
|
Allocation allocation = super.allocate(
|
||||||
applicationAttemptId, askCopy, release, blacklistAdditions,
|
applicationAttemptId, askCopy, release, blacklistAdditions,
|
||||||
blacklistRemovals, increaseRequests, decreaseRequests);
|
blacklistRemovals, updateRequests);
|
||||||
if (forceResourceLimit != null) {
|
if (forceResourceLimit != null) {
|
||||||
// Test wants to force the non-default resource limit
|
// Test wants to force the non-default resource limit
|
||||||
allocation.setResourceLimit(forceResourceLimit);
|
allocation.setResourceLimit(forceResourceLimit);
|
||||||
@ -1754,8 +1753,7 @@ public synchronized Allocation allocate(
|
|||||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||||
List<ContainerId> release, List<String> blacklistAdditions,
|
List<ContainerId> release, List<String> blacklistAdditions,
|
||||||
List<String> blacklistRemovals,
|
List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequest,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||||
for (ResourceRequest req : ask) {
|
for (ResourceRequest req : ask) {
|
||||||
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
ResourceRequest reqCopy = ResourceRequest.newInstance(req
|
||||||
@ -1766,7 +1764,7 @@ public synchronized Allocation allocate(
|
|||||||
SecurityUtil.setTokenServiceUseIp(false);
|
SecurityUtil.setTokenServiceUseIp(false);
|
||||||
Allocation normalAlloc = super.allocate(
|
Allocation normalAlloc = super.allocate(
|
||||||
applicationAttemptId, askCopy, release,
|
applicationAttemptId, askCopy, release,
|
||||||
blacklistAdditions, blacklistRemovals, null, null);
|
blacklistAdditions, blacklistRemovals, updateRequests);
|
||||||
List<Container> containers = normalAlloc.getContainers();
|
List<Container> containers = normalAlloc.getContainers();
|
||||||
if(containers.size() > 0) {
|
if(containers.size() > 0) {
|
||||||
// allocate excess container
|
// allocate excess container
|
||||||
|
@ -59,7 +59,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
||||||
@ -68,6 +67,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
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.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
||||||
@ -206,14 +206,13 @@ public void run() {
|
|||||||
public Allocation allocate(ApplicationAttemptId attemptId,
|
public Allocation allocate(ApplicationAttemptId attemptId,
|
||||||
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
|
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
|
||||||
List<String> strings, List<String> strings2,
|
List<String> strings, List<String> strings2,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
if (metricsON) {
|
if (metricsON) {
|
||||||
final Timer.Context context = schedulerAllocateTimer.time();
|
final Timer.Context context = schedulerAllocateTimer.time();
|
||||||
Allocation allocation = null;
|
Allocation allocation = null;
|
||||||
try {
|
try {
|
||||||
allocation = scheduler.allocate(attemptId, resourceRequests,
|
allocation = scheduler.allocate(attemptId, resourceRequests,
|
||||||
containerIds, strings, strings2, null, null);
|
containerIds, strings, strings2, updateRequests);
|
||||||
return allocation;
|
return allocation;
|
||||||
} finally {
|
} finally {
|
||||||
context.stop();
|
context.stop();
|
||||||
@ -227,7 +226,7 @@ public Allocation allocate(ApplicationAttemptId attemptId,
|
|||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
return scheduler.allocate(attemptId,
|
return scheduler.allocate(attemptId,
|
||||||
resourceRequests, containerIds, strings, strings2, null, null);
|
resourceRequests, containerIds, strings, strings2, updateRequests);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -51,10 +51,10 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
|
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.SchedulerApplication;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
||||||
@ -182,15 +182,14 @@ public void run() {
|
|||||||
public Allocation allocate(ApplicationAttemptId attemptId,
|
public Allocation allocate(ApplicationAttemptId attemptId,
|
||||||
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
|
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
|
||||||
List<String> strings, List<String> strings2,
|
List<String> strings, List<String> strings2,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
if (metricsON) {
|
if (metricsON) {
|
||||||
final Timer.Context context = schedulerAllocateTimer.time();
|
final Timer.Context context = schedulerAllocateTimer.time();
|
||||||
Allocation allocation = null;
|
Allocation allocation = null;
|
||||||
try {
|
try {
|
||||||
allocation = super
|
allocation = super
|
||||||
.allocate(attemptId, resourceRequests, containerIds, strings,
|
.allocate(attemptId, resourceRequests, containerIds, strings,
|
||||||
strings2, increaseRequests, decreaseRequests);
|
strings2, updateRequests);
|
||||||
return allocation;
|
return allocation;
|
||||||
} finally {
|
} finally {
|
||||||
context.stop();
|
context.stop();
|
||||||
@ -204,7 +203,7 @@ public Allocation allocate(ApplicationAttemptId attemptId,
|
|||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
return super.allocate(attemptId, resourceRequests, containerIds, strings,
|
return super.allocate(attemptId, resourceRequests, containerIds, strings,
|
||||||
strings2, increaseRequests, decreaseRequests);
|
strings2, updateRequests);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -39,7 +39,12 @@ public enum ContainerUpdateType {
|
|||||||
DECREASE_RESOURCE,
|
DECREASE_RESOURCE,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execution Type change.
|
* Execution Type promotion.
|
||||||
*/
|
*/
|
||||||
UPDATE_EXECUTION_TYPE
|
PROMOTE_EXECUTION_TYPE,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execution Type demotion.
|
||||||
|
*/
|
||||||
|
DEMOTE_EXECUTION_TYPE
|
||||||
}
|
}
|
||||||
|
@ -86,6 +86,12 @@ public int hashCode() {
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "UpdateContainerError{reason=" + getReason() + ", "
|
||||||
|
+ "req=" + getUpdateContainerRequest() + "}";
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object obj) {
|
public boolean equals(Object obj) {
|
||||||
if (this == obj) {
|
if (this == obj) {
|
||||||
|
@ -150,11 +150,13 @@ public int hashCode() {
|
|||||||
ContainerId cId = getContainerId();
|
ContainerId cId = getContainerId();
|
||||||
ExecutionType execType = getExecutionType();
|
ExecutionType execType = getExecutionType();
|
||||||
Resource capability = getCapability();
|
Resource capability = getCapability();
|
||||||
|
ContainerUpdateType updateType = getContainerUpdateType();
|
||||||
result =
|
result =
|
||||||
prime * result + ((capability == null) ? 0 : capability.hashCode());
|
prime * result + ((capability == null) ? 0 : capability.hashCode());
|
||||||
result = prime * result + ((cId == null) ? 0 : cId.hashCode());
|
result = prime * result + ((cId == null) ? 0 : cId.hashCode());
|
||||||
result = prime * result + getContainerVersion();
|
result = prime * result + getContainerVersion();
|
||||||
result = prime * result + ((execType == null) ? 0 : execType.hashCode());
|
result = prime * result + ((execType == null) ? 0 : execType.hashCode());
|
||||||
|
result = prime * result + ((updateType== null) ? 0 : updateType.hashCode());
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -208,6 +210,14 @@ public boolean equals(Object obj) {
|
|||||||
} else if (!execType.equals(other.getExecutionType())) {
|
} else if (!execType.equals(other.getExecutionType())) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
ContainerUpdateType updateType = getContainerUpdateType();
|
||||||
|
if (updateType == null) {
|
||||||
|
if (other.getContainerUpdateType() != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} else if (!updateType.equals(other.getContainerUpdateType())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -63,7 +63,8 @@ message FinishApplicationMasterResponseProto {
|
|||||||
enum ContainerUpdateTypeProto {
|
enum ContainerUpdateTypeProto {
|
||||||
INCREASE_RESOURCE = 0;
|
INCREASE_RESOURCE = 0;
|
||||||
DECREASE_RESOURCE = 1;
|
DECREASE_RESOURCE = 1;
|
||||||
UPDATE_EXECUTION_TYPE = 2;
|
PROMOTE_EXECUTION_TYPE = 2;
|
||||||
|
DEMOTE_EXECUTION_TYPE = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message UpdateContainerRequestProto {
|
message UpdateContainerRequestProto {
|
||||||
|
@ -66,6 +66,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
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.rmapp.attempt.RMAppAttemptState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
||||||
@ -574,8 +575,7 @@ public synchronized Allocation allocate(
|
|||||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||||
List<ContainerId> release, List<String> blacklistAdditions,
|
List<ContainerId> release, List<String> blacklistAdditions,
|
||||||
List<String> blacklistRemovals,
|
List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
|
||||||
for (ResourceRequest req : ask) {
|
for (ResourceRequest req : ask) {
|
||||||
ResourceRequest reqCopy =
|
ResourceRequest reqCopy =
|
||||||
@ -586,13 +586,12 @@ public synchronized Allocation allocate(
|
|||||||
}
|
}
|
||||||
lastAsk = ask;
|
lastAsk = ask;
|
||||||
lastRelease = release;
|
lastRelease = release;
|
||||||
lastIncrease = increaseRequests;
|
lastIncrease = updateRequests.getIncreaseRequests();
|
||||||
lastDecrease = decreaseRequests;
|
lastDecrease = updateRequests.getDecreaseRequests();
|
||||||
lastBlacklistAdditions = blacklistAdditions;
|
lastBlacklistAdditions = blacklistAdditions;
|
||||||
lastBlacklistRemovals = blacklistRemovals;
|
lastBlacklistRemovals = blacklistRemovals;
|
||||||
return super.allocate(applicationAttemptId, askCopy, release,
|
return super.allocate(applicationAttemptId, askCopy, release,
|
||||||
blacklistAdditions, blacklistRemovals, increaseRequests,
|
blacklistAdditions, blacklistRemovals, updateRequests);
|
||||||
decreaseRequests);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -321,13 +321,21 @@ private Container buildContainer(long rmIdentifier,
|
|||||||
// before accepting an ask)
|
// before accepting an ask)
|
||||||
Resource capability = normalizeCapability(appParams, rr);
|
Resource capability = normalizeCapability(appParams, rr);
|
||||||
|
|
||||||
|
return createContainer(
|
||||||
|
rmIdentifier, appParams.getContainerTokenExpiryInterval(),
|
||||||
|
SchedulerRequestKey.create(rr), userName, node, cId, capability);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Container createContainer(long rmIdentifier, long tokenExpiry,
|
||||||
|
SchedulerRequestKey schedulerKey, String userName, RemoteNode node,
|
||||||
|
ContainerId cId, Resource capability) {
|
||||||
long currTime = System.currentTimeMillis();
|
long currTime = System.currentTimeMillis();
|
||||||
ContainerTokenIdentifier containerTokenIdentifier =
|
ContainerTokenIdentifier containerTokenIdentifier =
|
||||||
new ContainerTokenIdentifier(
|
new ContainerTokenIdentifier(
|
||||||
cId, 0, node.getNodeId().toString(), userName,
|
cId, 0, node.getNodeId().toString(), userName,
|
||||||
capability, currTime + appParams.containerTokenExpiryInterval,
|
capability, currTime + tokenExpiry,
|
||||||
tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
|
tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
|
||||||
rr.getPriority(), currTime,
|
schedulerKey.getPriority(), currTime,
|
||||||
null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
|
null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
|
||||||
ExecutionType.OPPORTUNISTIC);
|
ExecutionType.OPPORTUNISTIC);
|
||||||
byte[] pwd =
|
byte[] pwd =
|
||||||
@ -336,9 +344,9 @@ private Container buildContainer(long rmIdentifier,
|
|||||||
containerTokenIdentifier);
|
containerTokenIdentifier);
|
||||||
Container container = BuilderUtils.newContainer(
|
Container container = BuilderUtils.newContainer(
|
||||||
cId, node.getNodeId(), node.getHttpAddress(),
|
cId, node.getNodeId(), node.getHttpAddress(),
|
||||||
capability, rr.getPriority(), containerToken,
|
capability, schedulerKey.getPriority(), containerToken,
|
||||||
containerTokenIdentifier.getExecutionType(),
|
containerTokenIdentifier.getExecutionType(),
|
||||||
rr.getAllocationRequestId());
|
schedulerKey.getAllocationRequestId());
|
||||||
return container;
|
return container;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -150,8 +150,9 @@ public void addToOutstandingReqs(List<ResourceRequest> resourceAsks) {
|
|||||||
resourceRequest.getNumContainers() + request.getNumContainers());
|
resourceRequest.getNumContainers() + request.getNumContainers());
|
||||||
}
|
}
|
||||||
if (ResourceRequest.isAnyLocation(request.getResourceName())) {
|
if (ResourceRequest.isAnyLocation(request.getResourceName())) {
|
||||||
LOG.info("# of outstandingOpReqs in ANY (at" +
|
LOG.info("# of outstandingOpReqs in ANY (at "
|
||||||
"priority = "+ schedulerKey.getPriority()
|
+ "priority = " + schedulerKey.getPriority()
|
||||||
|
+ ", allocationReqId = " + schedulerKey.getAllocationRequestId()
|
||||||
+ ", with capability = " + request.getCapability() + " ) : "
|
+ ", with capability = " + request.getCapability() + " ) : "
|
||||||
+ resourceRequest.getNumContainers());
|
+ resourceRequest.getNumContainers());
|
||||||
}
|
}
|
||||||
@ -167,7 +168,8 @@ public void addToOutstandingReqs(List<ResourceRequest> resourceAsks) {
|
|||||||
public void matchAllocationToOutstandingRequest(Resource capability,
|
public void matchAllocationToOutstandingRequest(Resource capability,
|
||||||
List<Container> allocatedContainers) {
|
List<Container> allocatedContainers) {
|
||||||
for (Container c : allocatedContainers) {
|
for (Container c : allocatedContainers) {
|
||||||
SchedulerRequestKey schedulerKey = SchedulerRequestKey.extractFrom(c);
|
SchedulerRequestKey schedulerKey =
|
||||||
|
SchedulerRequestKey.extractFrom(c);
|
||||||
Map<Resource, ResourceRequest> asks =
|
Map<Resource, ResourceRequest> asks =
|
||||||
outstandingOpReqs.get(schedulerKey);
|
outstandingOpReqs.get(schedulerKey);
|
||||||
|
|
||||||
|
@ -18,9 +18,12 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.yarn.server.scheduler;
|
package org.apache.hadoop.yarn.server.scheduler;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.api.records.AbstractResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Composite key for outstanding scheduler requests for any schedulable entity.
|
* Composite key for outstanding scheduler requests for any schedulable entity.
|
||||||
@ -31,6 +34,7 @@ public final class SchedulerRequestKey implements
|
|||||||
|
|
||||||
private final Priority priority;
|
private final Priority priority;
|
||||||
private final long allocationRequestId;
|
private final long allocationRequestId;
|
||||||
|
private final ContainerId containerToUpdate;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory method to generate a SchedulerRequestKey from a ResourceRequest.
|
* Factory method to generate a SchedulerRequestKey from a ResourceRequest.
|
||||||
@ -39,7 +43,13 @@ public final class SchedulerRequestKey implements
|
|||||||
*/
|
*/
|
||||||
public static SchedulerRequestKey create(ResourceRequest req) {
|
public static SchedulerRequestKey create(ResourceRequest req) {
|
||||||
return new SchedulerRequestKey(req.getPriority(),
|
return new SchedulerRequestKey(req.getPriority(),
|
||||||
req.getAllocationRequestId());
|
req.getAllocationRequestId(), null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static SchedulerRequestKey create(UpdateContainerRequest req,
|
||||||
|
SchedulerRequestKey schedulerRequestKey) {
|
||||||
|
return new SchedulerRequestKey(schedulerRequestKey.getPriority(),
|
||||||
|
schedulerRequestKey.getAllocationRequestId(), req.getContainerId());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -50,12 +60,16 @@ public static SchedulerRequestKey create(ResourceRequest req) {
|
|||||||
*/
|
*/
|
||||||
public static SchedulerRequestKey extractFrom(Container container) {
|
public static SchedulerRequestKey extractFrom(Container container) {
|
||||||
return new SchedulerRequestKey(container.getPriority(),
|
return new SchedulerRequestKey(container.getPriority(),
|
||||||
container.getAllocationRequestId());
|
container.getAllocationRequestId(), null);
|
||||||
}
|
}
|
||||||
|
|
||||||
SchedulerRequestKey(Priority priority, long allocationRequestId) {
|
|
||||||
|
|
||||||
|
public SchedulerRequestKey(Priority priority, long allocationRequestId,
|
||||||
|
ContainerId containerToUpdate) {
|
||||||
this.priority = priority;
|
this.priority = priority;
|
||||||
this.allocationRequestId = allocationRequestId;
|
this.allocationRequestId = allocationRequestId;
|
||||||
|
this.containerToUpdate = containerToUpdate;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -76,6 +90,10 @@ public long getAllocationRequestId() {
|
|||||||
return allocationRequestId;
|
return allocationRequestId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ContainerId getContainerToUpdate() {
|
||||||
|
return containerToUpdate;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compareTo(SchedulerRequestKey o) {
|
public int compareTo(SchedulerRequestKey o) {
|
||||||
if (o == null) {
|
if (o == null) {
|
||||||
@ -85,6 +103,15 @@ public int compareTo(SchedulerRequestKey o) {
|
|||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Ensure updates are ranked higher
|
||||||
|
if (this.containerToUpdate == null && o.containerToUpdate != null) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
if (this.containerToUpdate != null && o.containerToUpdate == null) {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
int priorityCompare = o.getPriority().compareTo(priority);
|
int priorityCompare = o.getPriority().compareTo(priority);
|
||||||
// we first sort by priority and then by allocationRequestId
|
// we first sort by priority and then by allocationRequestId
|
||||||
if (priorityCompare != 0) {
|
if (priorityCompare != 0) {
|
||||||
@ -107,16 +134,21 @@ public boolean equals(Object o) {
|
|||||||
if (getAllocationRequestId() != that.getAllocationRequestId()) {
|
if (getAllocationRequestId() != that.getAllocationRequestId()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
return getPriority() != null ?
|
if (!getPriority().equals(that.getPriority())) {
|
||||||
getPriority().equals(that.getPriority()) :
|
return false;
|
||||||
that.getPriority() == null;
|
}
|
||||||
|
return containerToUpdate != null ?
|
||||||
|
containerToUpdate.equals(that.containerToUpdate) :
|
||||||
|
that.containerToUpdate == null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
int result = getPriority() != null ? getPriority().hashCode() : 0;
|
int result = priority != null ? priority.hashCode() : 0;
|
||||||
result = 31 * result + (int) (getAllocationRequestId() ^ (
|
result = 31 * result + (int) (allocationRequestId ^ (allocationRequestId
|
||||||
getAllocationRequestId() >>> 32));
|
>>> 32));
|
||||||
|
result = 31 * result + (containerToUpdate != null ? containerToUpdate
|
||||||
|
.hashCode() : 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -125,6 +157,7 @@ public String toString() {
|
|||||||
return "SchedulerRequestKey{" +
|
return "SchedulerRequestKey{" +
|
||||||
"priority=" + priority +
|
"priority=" + priority +
|
||||||
", allocationRequestId=" + allocationRequestId +
|
", allocationRequestId=" + allocationRequestId +
|
||||||
|
", containerToUpdate=" + containerToUpdate +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -68,7 +68,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
|
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
|
import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
|
||||||
@ -93,7 +92,10 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
||||||
|
.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security
|
import org.apache.hadoop.yarn.server.resourcemanager.security
|
||||||
@ -559,12 +561,10 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
// Split Update Resource Requests into increase and decrease.
|
// Split Update Resource Requests into increase and decrease.
|
||||||
// No Exceptions are thrown here. All update errors are aggregated
|
// No Exceptions are thrown here. All update errors are aggregated
|
||||||
// and returned to the AM.
|
// and returned to the AM.
|
||||||
List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
|
List<UpdateContainerError> updateErrors = new ArrayList<>();
|
||||||
List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
|
ContainerUpdates containerUpdateRequests =
|
||||||
List<UpdateContainerError> updateContainerErrors =
|
|
||||||
RMServerUtils.validateAndSplitUpdateResourceRequests(
|
RMServerUtils.validateAndSplitUpdateResourceRequests(
|
||||||
rmContext, request, maximumCapacity,
|
rmContext, request, maximumCapacity, updateErrors);
|
||||||
increaseResourceReqs, decreaseResourceReqs);
|
|
||||||
|
|
||||||
// Send new requests to appAttempt.
|
// Send new requests to appAttempt.
|
||||||
Allocation allocation;
|
Allocation allocation;
|
||||||
@ -580,7 +580,7 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
allocation =
|
allocation =
|
||||||
this.rScheduler.allocate(appAttemptId, ask, release,
|
this.rScheduler.allocate(appAttemptId, ask, release,
|
||||||
blacklistAdditions, blacklistRemovals,
|
blacklistAdditions, blacklistRemovals,
|
||||||
increaseResourceReqs, decreaseResourceReqs);
|
containerUpdateRequests);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
|
if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
|
||||||
@ -596,7 +596,7 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Notify the AM of container update errors
|
// Notify the AM of container update errors
|
||||||
addToUpdateContainerErrors(allocateResponse, updateContainerErrors);
|
addToUpdateContainerErrors(allocateResponse, updateErrors);
|
||||||
|
|
||||||
// update the response with the deltas of node status changes
|
// update the response with the deltas of node status changes
|
||||||
List<RMNode> updatedNodes = new ArrayList<RMNode>();
|
List<RMNode> updatedNodes = new ArrayList<RMNode>();
|
||||||
@ -630,15 +630,7 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
.pullJustFinishedContainers());
|
.pullJustFinishedContainers());
|
||||||
allocateResponse.setAvailableResources(allocation.getResourceLimit());
|
allocateResponse.setAvailableResources(allocation.getResourceLimit());
|
||||||
|
|
||||||
// Handling increased containers
|
addToContainerUpdates(appAttemptId, allocateResponse, allocation);
|
||||||
addToUpdatedContainers(
|
|
||||||
allocateResponse, ContainerUpdateType.INCREASE_RESOURCE,
|
|
||||||
allocation.getIncreasedContainers());
|
|
||||||
|
|
||||||
// Handling decreased containers
|
|
||||||
addToUpdatedContainers(
|
|
||||||
allocateResponse, ContainerUpdateType.DECREASE_RESOURCE,
|
|
||||||
allocation.getDecreasedContainers());
|
|
||||||
|
|
||||||
allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
|
allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
|
||||||
|
|
||||||
@ -658,6 +650,33 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
.getApplicationPriority());
|
.getApplicationPriority());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void addToContainerUpdates(ApplicationAttemptId appAttemptId,
|
||||||
|
AllocateResponse allocateResponse, Allocation allocation) {
|
||||||
|
// Handling increased containers
|
||||||
|
addToUpdatedContainers(
|
||||||
|
allocateResponse, ContainerUpdateType.INCREASE_RESOURCE,
|
||||||
|
allocation.getIncreasedContainers());
|
||||||
|
|
||||||
|
// Handling decreased containers
|
||||||
|
addToUpdatedContainers(
|
||||||
|
allocateResponse, ContainerUpdateType.DECREASE_RESOURCE,
|
||||||
|
allocation.getDecreasedContainers());
|
||||||
|
|
||||||
|
// Handling promoted containers
|
||||||
|
addToUpdatedContainers(
|
||||||
|
allocateResponse, ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
allocation.getPromotedContainers());
|
||||||
|
|
||||||
|
// Handling demoted containers
|
||||||
|
addToUpdatedContainers(
|
||||||
|
allocateResponse, ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
|
||||||
|
allocation.getDemotedContainers());
|
||||||
|
|
||||||
|
addToUpdateContainerErrors(allocateResponse,
|
||||||
|
((AbstractYarnScheduler)rScheduler)
|
||||||
|
.getApplicationAttempt(appAttemptId).pullUpdateContainerErrors());
|
||||||
|
}
|
||||||
|
|
||||||
protected void addToUpdateContainerErrors(AllocateResponse allocateResponse,
|
protected void addToUpdateContainerErrors(AllocateResponse allocateResponse,
|
||||||
List<UpdateContainerError> updateContainerErrors) {
|
List<UpdateContainerError> updateContainerErrors) {
|
||||||
if (!updateContainerErrors.isEmpty()) {
|
if (!updateContainerErrors.isEmpty()) {
|
||||||
|
@ -32,7 +32,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
|
||||||
import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
|
import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
|
||||||
@ -48,7 +47,6 @@
|
|||||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||||
import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
|
import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RemoteNode;
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RemoteNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
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.RMContainerEvent;
|
||||||
@ -57,6 +55,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
|
||||||
|
|
||||||
@ -69,9 +68,9 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
|
import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@ -251,6 +250,7 @@ protected void allocateInternal(ApplicationAttemptId appAttemptId,
|
|||||||
|
|
||||||
// Allocate GUARANTEED containers.
|
// Allocate GUARANTEED containers.
|
||||||
request.setAskList(partitionedAsks.getGuaranteed());
|
request.setAskList(partitionedAsks.getGuaranteed());
|
||||||
|
|
||||||
super.allocateInternal(appAttemptId, request, allocateResponse);
|
super.allocateInternal(appAttemptId, request, allocateResponse);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -298,15 +298,9 @@ private void handleNewContainers(List<Container> allocContainers,
|
|||||||
boolean isRemotelyAllocated) {
|
boolean isRemotelyAllocated) {
|
||||||
for (Container container : allocContainers) {
|
for (Container container : allocContainers) {
|
||||||
// Create RMContainer
|
// Create RMContainer
|
||||||
SchedulerApplicationAttempt appAttempt =
|
RMContainer rmContainer =
|
||||||
((AbstractYarnScheduler) rmContext.getScheduler())
|
SchedulerUtils.createOpportunisticRmContainer(
|
||||||
.getCurrentAttemptForContainer(container.getId());
|
rmContext, container, isRemotelyAllocated);
|
||||||
RMContainer rmContainer = new RMContainerImpl(container,
|
|
||||||
appAttempt.getApplicationAttemptId(), container.getNodeId(),
|
|
||||||
appAttempt.getUser(), rmContext, isRemotelyAllocated);
|
|
||||||
appAttempt.addRMContainer(container.getId(), rmContainer);
|
|
||||||
((AbstractYarnScheduler) rmContext.getScheduler()).getNode(
|
|
||||||
container.getNodeId()).allocateContainer(rmContainer);
|
|
||||||
rmContainer.handle(
|
rmContainer.handle(
|
||||||
new RMContainerEvent(container.getId(),
|
new RMContainerEvent(container.getId(),
|
||||||
RMContainerEventType.ACQUIRED));
|
RMContainerEventType.ACQUIRED));
|
||||||
|
@ -39,6 +39,8 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeState;
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
||||||
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
@ -63,6 +65,7 @@
|
|||||||
.RMAppAttemptState;
|
.RMAppAttemptState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
||||||
.ResourceScheduler;
|
.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler
|
||||||
@ -80,7 +83,7 @@
|
|||||||
*/
|
*/
|
||||||
public class RMServerUtils {
|
public class RMServerUtils {
|
||||||
|
|
||||||
private static final String UPDATE_OUTSTANDING_ERROR =
|
public static final String UPDATE_OUTSTANDING_ERROR =
|
||||||
"UPDATE_OUTSTANDING_ERROR";
|
"UPDATE_OUTSTANDING_ERROR";
|
||||||
private static final String INCORRECT_CONTAINER_VERSION_ERROR =
|
private static final String INCORRECT_CONTAINER_VERSION_ERROR =
|
||||||
"INCORRECT_CONTAINER_VERSION_ERROR";
|
"INCORRECT_CONTAINER_VERSION_ERROR";
|
||||||
@ -124,74 +127,105 @@ public static List<RMNode> queryRMNodes(RMContext context,
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if we have:
|
* Check if we have:
|
||||||
* - Request for same containerId and different target resource
|
* - Request for same containerId and different target resource.
|
||||||
* - If targetResources violates maximum/minimumAllocation
|
* - If targetResources violates maximum/minimumAllocation.
|
||||||
* @param rmContext RM context
|
* @param rmContext RM context.
|
||||||
* @param request Allocate Request
|
* @param request Allocate Request.
|
||||||
* @param maximumAllocation Maximum Allocation
|
* @param maximumAllocation Maximum Allocation.
|
||||||
* @param increaseResourceReqs Increase Resource Request
|
* @param updateErrors Container update errors.
|
||||||
* @param decreaseResourceReqs Decrease Resource Request
|
* @return ContainerUpdateRequests.
|
||||||
* @return List of container Errors
|
|
||||||
*/
|
*/
|
||||||
public static List<UpdateContainerError>
|
public static ContainerUpdates
|
||||||
validateAndSplitUpdateResourceRequests(RMContext rmContext,
|
validateAndSplitUpdateResourceRequests(RMContext rmContext,
|
||||||
AllocateRequest request, Resource maximumAllocation,
|
AllocateRequest request, Resource maximumAllocation,
|
||||||
List<UpdateContainerRequest> increaseResourceReqs,
|
List<UpdateContainerError> updateErrors) {
|
||||||
List<UpdateContainerRequest> decreaseResourceReqs) {
|
ContainerUpdates updateRequests =
|
||||||
List<UpdateContainerError> errors = new ArrayList<>();
|
new ContainerUpdates();
|
||||||
Set<ContainerId> outstandingUpdate = new HashSet<>();
|
Set<ContainerId> outstandingUpdate = new HashSet<>();
|
||||||
for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
|
for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
|
||||||
RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
|
RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
|
||||||
updateReq.getContainerId());
|
updateReq.getContainerId());
|
||||||
String msg = null;
|
String msg = validateContainerIdAndVersion(outstandingUpdate,
|
||||||
if (rmContainer == null) {
|
updateReq, rmContainer);
|
||||||
msg = INVALID_CONTAINER_ID;
|
ContainerUpdateType updateType = updateReq.getContainerUpdateType();
|
||||||
}
|
|
||||||
// Only allow updates if the requested version matches the current
|
|
||||||
// version
|
|
||||||
if (msg == null && updateReq.getContainerVersion() !=
|
|
||||||
rmContainer.getContainer().getVersion()) {
|
|
||||||
msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
|
|
||||||
+ updateReq.getContainerVersion() + "|"
|
|
||||||
+ rmContainer.getContainer().getVersion();
|
|
||||||
}
|
|
||||||
// No more than 1 container update per request.
|
|
||||||
if (msg == null &&
|
|
||||||
outstandingUpdate.contains(updateReq.getContainerId())) {
|
|
||||||
msg = UPDATE_OUTSTANDING_ERROR;
|
|
||||||
}
|
|
||||||
if (msg == null) {
|
if (msg == null) {
|
||||||
Resource original = rmContainer.getContainer().getResource();
|
if ((updateType != ContainerUpdateType.PROMOTE_EXECUTION_TYPE) &&
|
||||||
Resource target = updateReq.getCapability();
|
(updateType !=ContainerUpdateType.DEMOTE_EXECUTION_TYPE)) {
|
||||||
if (Resources.fitsIn(target, original)) {
|
Resource original = rmContainer.getContainer().getResource();
|
||||||
// This is a decrease request
|
Resource target = updateReq.getCapability();
|
||||||
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
|
if (Resources.fitsIn(target, original)) {
|
||||||
maximumAllocation, false)) {
|
// This is a decrease request
|
||||||
decreaseResourceReqs.add(updateReq);
|
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
|
||||||
outstandingUpdate.add(updateReq.getContainerId());
|
maximumAllocation, false)) {
|
||||||
|
updateRequests.getDecreaseRequests().add(updateReq);
|
||||||
|
outstandingUpdate.add(updateReq.getContainerId());
|
||||||
|
} else {
|
||||||
|
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
|
// This is an increase request
|
||||||
|
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
|
||||||
|
maximumAllocation, true)) {
|
||||||
|
updateRequests.getIncreaseRequests().add(updateReq);
|
||||||
|
outstandingUpdate.add(updateReq.getContainerId());
|
||||||
|
} else {
|
||||||
|
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// This is an increase request
|
ExecutionType original = rmContainer.getExecutionType();
|
||||||
if (validateIncreaseDecreaseRequest(rmContext, updateReq,
|
ExecutionType target = updateReq.getExecutionType();
|
||||||
maximumAllocation, true)) {
|
if (target != original) {
|
||||||
increaseResourceReqs.add(updateReq);
|
if (target == ExecutionType.GUARANTEED &&
|
||||||
outstandingUpdate.add(updateReq.getContainerId());
|
original == ExecutionType.OPPORTUNISTIC) {
|
||||||
} else {
|
updateRequests.getPromotionRequests().add(updateReq);
|
||||||
msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
|
outstandingUpdate.add(updateReq.getContainerId());
|
||||||
|
} else if (target == ExecutionType.OPPORTUNISTIC &&
|
||||||
|
original == ExecutionType.GUARANTEED) {
|
||||||
|
updateRequests.getDemotionRequests().add(updateReq);
|
||||||
|
outstandingUpdate.add(updateReq.getContainerId());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (msg != null) {
|
checkAndcreateUpdateError(updateErrors, updateReq, msg);
|
||||||
UpdateContainerError updateError = RECORD_FACTORY
|
|
||||||
.newRecordInstance(UpdateContainerError.class);
|
|
||||||
updateError.setReason(msg);
|
|
||||||
updateError.setUpdateContainerRequest(updateReq);
|
|
||||||
errors.add(updateError);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return errors;
|
return updateRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void checkAndcreateUpdateError(
|
||||||
|
List<UpdateContainerError> errors, UpdateContainerRequest updateReq,
|
||||||
|
String msg) {
|
||||||
|
if (msg != null) {
|
||||||
|
UpdateContainerError updateError = RECORD_FACTORY
|
||||||
|
.newRecordInstance(UpdateContainerError.class);
|
||||||
|
updateError.setReason(msg);
|
||||||
|
updateError.setUpdateContainerRequest(updateReq);
|
||||||
|
errors.add(updateError);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String validateContainerIdAndVersion(
|
||||||
|
Set<ContainerId> outstandingUpdate, UpdateContainerRequest updateReq,
|
||||||
|
RMContainer rmContainer) {
|
||||||
|
String msg = null;
|
||||||
|
if (rmContainer == null) {
|
||||||
|
msg = INVALID_CONTAINER_ID;
|
||||||
|
}
|
||||||
|
// Only allow updates if the requested version matches the current
|
||||||
|
// version
|
||||||
|
if (msg == null && updateReq.getContainerVersion() !=
|
||||||
|
rmContainer.getContainer().getVersion()) {
|
||||||
|
msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
|
||||||
|
+ updateReq.getContainerVersion() + "|"
|
||||||
|
+ rmContainer.getContainer().getVersion();
|
||||||
|
}
|
||||||
|
// No more than 1 container update per request.
|
||||||
|
if (msg == null &&
|
||||||
|
outstandingUpdate.contains(updateReq.getContainerId())) {
|
||||||
|
msg = UPDATE_OUTSTANDING_ERROR;
|
||||||
|
}
|
||||||
|
return msg;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -95,6 +95,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
@ -1072,7 +1073,8 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
|
|||||||
Collections.singletonList(appAttempt.amReq),
|
Collections.singletonList(appAttempt.amReq),
|
||||||
EMPTY_CONTAINER_RELEASE_LIST,
|
EMPTY_CONTAINER_RELEASE_LIST,
|
||||||
amBlacklist.getBlacklistAdditions(),
|
amBlacklist.getBlacklistAdditions(),
|
||||||
amBlacklist.getBlacklistRemovals(), null, null);
|
amBlacklist.getBlacklistRemovals(),
|
||||||
|
new ContainerUpdates());
|
||||||
if (amContainerAllocation != null
|
if (amContainerAllocation != null
|
||||||
&& amContainerAllocation.getContainers() != null) {
|
&& amContainerAllocation.getContainers() != null) {
|
||||||
assert (amContainerAllocation.getContainers().size() == 0);
|
assert (amContainerAllocation.getContainers().size() == 0);
|
||||||
@ -1096,7 +1098,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
|
|||||||
Allocation amContainerAllocation =
|
Allocation amContainerAllocation =
|
||||||
appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
|
appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
|
||||||
EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
|
EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
|
||||||
null, null, null);
|
null, new ContainerUpdates());
|
||||||
// There must be at least one container allocated, because a
|
// There must be at least one container allocated, because a
|
||||||
// CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
|
// CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
|
||||||
// and is put in SchedulerApplication#newlyAllocatedContainers.
|
// and is put in SchedulerApplication#newlyAllocatedContainers.
|
||||||
|
@ -108,6 +108,8 @@ RMContainerEventType.KILL, new FinishedTransition())
|
|||||||
// Transitions from ACQUIRED state
|
// Transitions from ACQUIRED state
|
||||||
.addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
|
.addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
|
||||||
RMContainerEventType.LAUNCHED)
|
RMContainerEventType.LAUNCHED)
|
||||||
|
.addTransition(RMContainerState.ACQUIRED, RMContainerState.ACQUIRED,
|
||||||
|
RMContainerEventType.ACQUIRED)
|
||||||
.addTransition(RMContainerState.ACQUIRED, RMContainerState.COMPLETED,
|
.addTransition(RMContainerState.ACQUIRED, RMContainerState.COMPLETED,
|
||||||
RMContainerEventType.FINISHED, new FinishedTransition())
|
RMContainerEventType.FINISHED, new FinishedTransition())
|
||||||
.addTransition(RMContainerState.ACQUIRED, RMContainerState.RELEASED,
|
.addTransition(RMContainerState.ACQUIRED, RMContainerState.RELEASED,
|
||||||
@ -124,6 +126,8 @@ RMContainerEventType.FINISHED, new FinishedTransition())
|
|||||||
RMContainerEventType.KILL, new KillTransition())
|
RMContainerEventType.KILL, new KillTransition())
|
||||||
.addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED,
|
.addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED,
|
||||||
RMContainerEventType.RELEASED, new KillTransition())
|
RMContainerEventType.RELEASED, new KillTransition())
|
||||||
|
.addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
|
||||||
|
RMContainerEventType.ACQUIRED)
|
||||||
.addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
|
.addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
|
||||||
RMContainerEventType.RESERVED, new ContainerReservedTransition())
|
RMContainerEventType.RESERVED, new ContainerReservedTransition())
|
||||||
.addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
|
.addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
|
||||||
@ -163,13 +167,13 @@ RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition())
|
|||||||
private final WriteLock writeLock;
|
private final WriteLock writeLock;
|
||||||
private final ApplicationAttemptId appAttemptId;
|
private final ApplicationAttemptId appAttemptId;
|
||||||
private final NodeId nodeId;
|
private final NodeId nodeId;
|
||||||
private final Container container;
|
|
||||||
private final RMContext rmContext;
|
private final RMContext rmContext;
|
||||||
private final EventHandler eventHandler;
|
private final EventHandler eventHandler;
|
||||||
private final ContainerAllocationExpirer containerAllocationExpirer;
|
private final ContainerAllocationExpirer containerAllocationExpirer;
|
||||||
private final String user;
|
private final String user;
|
||||||
private final String nodeLabelExpression;
|
private final String nodeLabelExpression;
|
||||||
|
|
||||||
|
private volatile Container container;
|
||||||
private Resource reservedResource;
|
private Resource reservedResource;
|
||||||
private NodeId reservedNode;
|
private NodeId reservedNode;
|
||||||
private SchedulerRequestKey reservedSchedulerKey;
|
private SchedulerRequestKey reservedSchedulerKey;
|
||||||
@ -188,44 +192,44 @@ RMContainerEventType.CHANGE_RESOURCE, new ChangeResourceTransition())
|
|||||||
private boolean isExternallyAllocated;
|
private boolean isExternallyAllocated;
|
||||||
private SchedulerRequestKey allocatedSchedulerKey;
|
private SchedulerRequestKey allocatedSchedulerKey;
|
||||||
|
|
||||||
public RMContainerImpl(Container container,
|
public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
|
||||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||||
RMContext rmContext) {
|
RMContext rmContext) {
|
||||||
this(container, appAttemptId, nodeId, user, rmContext, System
|
this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
|
||||||
.currentTimeMillis(), "");
|
.currentTimeMillis(), "");
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainerImpl(Container container,
|
public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
|
||||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||||
RMContext rmContext, boolean isExternallyAllocated) {
|
RMContext rmContext, boolean isExternallyAllocated) {
|
||||||
this(container, appAttemptId, nodeId, user, rmContext, System
|
this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
|
||||||
.currentTimeMillis(), "", isExternallyAllocated);
|
.currentTimeMillis(), "", isExternallyAllocated);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean saveNonAMContainerMetaInfo;
|
private boolean saveNonAMContainerMetaInfo;
|
||||||
|
|
||||||
public RMContainerImpl(Container container,
|
public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
|
||||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||||
RMContext rmContext, String nodeLabelExpression) {
|
RMContext rmContext, String nodeLabelExpression) {
|
||||||
this(container, appAttemptId, nodeId, user, rmContext, System
|
this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
|
||||||
.currentTimeMillis(), nodeLabelExpression);
|
.currentTimeMillis(), nodeLabelExpression);
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainerImpl(Container container,
|
public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
|
||||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||||
RMContext rmContext, long creationTime, String nodeLabelExpression) {
|
RMContext rmContext, long creationTime, String nodeLabelExpression) {
|
||||||
this(container, appAttemptId, nodeId, user, rmContext, creationTime,
|
this(container, schedulerKey, appAttemptId, nodeId, user, rmContext,
|
||||||
nodeLabelExpression, false);
|
creationTime, nodeLabelExpression, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public RMContainerImpl(Container container,
|
public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
|
||||||
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
|
||||||
RMContext rmContext, long creationTime, String nodeLabelExpression,
|
RMContext rmContext, long creationTime, String nodeLabelExpression,
|
||||||
boolean isExternallyAllocated) {
|
boolean isExternallyAllocated) {
|
||||||
this.stateMachine = stateMachineFactory.make(this);
|
this.stateMachine = stateMachineFactory.make(this);
|
||||||
this.nodeId = nodeId;
|
this.nodeId = nodeId;
|
||||||
this.container = container;
|
this.container = container;
|
||||||
this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container);
|
this.allocatedSchedulerKey = schedulerKey;
|
||||||
this.appAttemptId = appAttemptId;
|
this.appAttemptId = appAttemptId;
|
||||||
this.user = user;
|
this.user = user;
|
||||||
this.creationTime = creationTime;
|
this.creationTime = creationTime;
|
||||||
@ -276,6 +280,10 @@ public Container getContainer() {
|
|||||||
return this.container;
|
return this.container;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setContainer(Container container) {
|
||||||
|
this.container = container;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RMContainerState getState() {
|
public RMContainerState getState() {
|
||||||
this.readLock.lock();
|
this.readLock.lock();
|
||||||
|
@ -51,6 +51,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
|
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
|
||||||
@ -62,6 +63,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
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.ResourceManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
||||||
@ -81,6 +83,11 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.server.utils.Lock;
|
import org.apache.hadoop.yarn.server.utils.Lock;
|
||||||
import org.apache.hadoop.yarn.util.Clock;
|
import org.apache.hadoop.yarn.util.Clock;
|
||||||
import org.apache.hadoop.yarn.util.SystemClock;
|
import org.apache.hadoop.yarn.util.SystemClock;
|
||||||
@ -504,9 +511,11 @@ private RMContainer recoverAndCreateContainer(NMContainerStatus status,
|
|||||||
ApplicationAttemptId attemptId =
|
ApplicationAttemptId attemptId =
|
||||||
container.getId().getApplicationAttemptId();
|
container.getId().getApplicationAttemptId();
|
||||||
RMContainer rmContainer =
|
RMContainer rmContainer =
|
||||||
new RMContainerImpl(container, attemptId, node.getNodeID(),
|
new RMContainerImpl(container,
|
||||||
applications.get(attemptId.getApplicationId()).getUser(), rmContext,
|
SchedulerRequestKey.extractFrom(container), attemptId,
|
||||||
status.getCreationTime(), status.getNodeLabelExpression());
|
node.getNodeID(), applications.get(
|
||||||
|
attemptId.getApplicationId()).getUser(), rmContext,
|
||||||
|
status.getCreationTime(), status.getNodeLabelExpression());
|
||||||
return rmContainer;
|
return rmContainer;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1053,4 +1062,93 @@ protected void normalizeRequests(List<ResourceRequest> asks) {
|
|||||||
normalizeRequest(ask);
|
normalizeRequest(ask);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void handleExecutionTypeUpdates(
|
||||||
|
SchedulerApplicationAttempt appAttempt,
|
||||||
|
List<UpdateContainerRequest> promotionRequests,
|
||||||
|
List<UpdateContainerRequest> demotionRequests) {
|
||||||
|
if (promotionRequests != null && !promotionRequests.isEmpty()) {
|
||||||
|
LOG.info("Promotion Update requests : " + promotionRequests);
|
||||||
|
handlePromotionRequests(appAttempt, promotionRequests);
|
||||||
|
}
|
||||||
|
if (demotionRequests != null && !demotionRequests.isEmpty()) {
|
||||||
|
LOG.info("Demotion Update requests : " + demotionRequests);
|
||||||
|
handleDemotionRequests(appAttempt, demotionRequests);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void handlePromotionRequests(
|
||||||
|
SchedulerApplicationAttempt applicationAttempt,
|
||||||
|
List<UpdateContainerRequest> updateContainerRequests) {
|
||||||
|
for (UpdateContainerRequest uReq : updateContainerRequests) {
|
||||||
|
RMContainer rmContainer =
|
||||||
|
rmContext.getScheduler().getRMContainer(uReq.getContainerId());
|
||||||
|
// Check if this is a container update
|
||||||
|
// And not in the middle of a Demotion
|
||||||
|
if (rmContainer != null) {
|
||||||
|
// Check if this is an executionType change request
|
||||||
|
// If so, fix the rr to make it look like a normal rr
|
||||||
|
// with relaxLocality=false and numContainers=1
|
||||||
|
SchedulerNode schedulerNode = rmContext.getScheduler()
|
||||||
|
.getSchedulerNode(rmContainer.getContainer().getNodeId());
|
||||||
|
|
||||||
|
// Add only if no outstanding promote requests exist.
|
||||||
|
if (!applicationAttempt.getUpdateContext()
|
||||||
|
.checkAndAddToOutstandingIncreases(
|
||||||
|
rmContainer, schedulerNode, uReq)) {
|
||||||
|
applicationAttempt.addToUpdateContainerErrors(
|
||||||
|
UpdateContainerError.newInstance(
|
||||||
|
RMServerUtils.UPDATE_OUTSTANDING_ERROR, uReq));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.warn("Cannot promote non-existent (or completed) Container ["
|
||||||
|
+ uReq.getContainerId() + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void handleDemotionRequests(SchedulerApplicationAttempt appAttempt,
|
||||||
|
List<UpdateContainerRequest> demotionRequests) {
|
||||||
|
OpportunisticContainerContext oppCntxt =
|
||||||
|
appAttempt.getOpportunisticContainerContext();
|
||||||
|
for (UpdateContainerRequest uReq : demotionRequests) {
|
||||||
|
RMContainer rmContainer =
|
||||||
|
rmContext.getScheduler().getRMContainer(uReq.getContainerId());
|
||||||
|
if (rmContainer != null) {
|
||||||
|
if (appAttempt.getUpdateContext().checkAndAddToOutstandingDecreases(
|
||||||
|
rmContainer.getContainer())) {
|
||||||
|
RMContainer demotedRMContainer =
|
||||||
|
createDemotedRMContainer(appAttempt, oppCntxt, rmContainer);
|
||||||
|
appAttempt.addToNewlyDemotedContainers(
|
||||||
|
uReq.getContainerId(), demotedRMContainer);
|
||||||
|
} else {
|
||||||
|
appAttempt.addToUpdateContainerErrors(
|
||||||
|
UpdateContainerError.newInstance(
|
||||||
|
RMServerUtils.UPDATE_OUTSTANDING_ERROR, uReq));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.warn("Cannot demote non-existent (or completed) Container ["
|
||||||
|
+ uReq.getContainerId() + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private RMContainer createDemotedRMContainer(
|
||||||
|
SchedulerApplicationAttempt appAttempt,
|
||||||
|
OpportunisticContainerContext oppCntxt,
|
||||||
|
RMContainer rmContainer) {
|
||||||
|
SchedulerRequestKey sk =
|
||||||
|
SchedulerRequestKey.extractFrom(rmContainer.getContainer());
|
||||||
|
Container demotedContainer = BuilderUtils.newContainer(
|
||||||
|
ContainerId.newContainerId(appAttempt.getApplicationAttemptId(),
|
||||||
|
oppCntxt.getContainerIdGenerator().generateContainerId()),
|
||||||
|
rmContainer.getContainer().getNodeId(),
|
||||||
|
rmContainer.getContainer().getNodeHttpAddress(),
|
||||||
|
rmContainer.getContainer().getResource(),
|
||||||
|
sk.getPriority(), null, ExecutionType.OPPORTUNISTIC,
|
||||||
|
sk.getAllocationRequestId());
|
||||||
|
demotedContainer.setVersion(rmContainer.getContainer().getVersion());
|
||||||
|
return SchedulerUtils.createOpportunisticRmContainer(
|
||||||
|
rmContext, demotedContainer, false);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -36,6 +36,8 @@ public class Allocation {
|
|||||||
final List<NMToken> nmTokens;
|
final List<NMToken> nmTokens;
|
||||||
final List<Container> increasedContainers;
|
final List<Container> increasedContainers;
|
||||||
final List<Container> decreasedContainers;
|
final List<Container> decreasedContainers;
|
||||||
|
final List<Container> promotedContainers;
|
||||||
|
final List<Container> demotedContainers;
|
||||||
private Resource resourceLimit;
|
private Resource resourceLimit;
|
||||||
|
|
||||||
|
|
||||||
@ -50,13 +52,23 @@ public Allocation(List<Container> containers, Resource resourceLimit,
|
|||||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
|
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
|
||||||
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
||||||
fungibleResources, nmTokens, null, null);
|
fungibleResources, nmTokens, null, null, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
|
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
|
||||||
List<Container> increasedContainers, List<Container> decreasedContainer) {
|
List<Container> increasedContainers, List<Container> decreasedContainer) {
|
||||||
|
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
||||||
|
fungibleResources, nmTokens, increasedContainers, decreasedContainer,
|
||||||
|
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,
|
||||||
|
List<Container> promotedContainers, List<Container> demotedContainer) {
|
||||||
this.containers = containers;
|
this.containers = containers;
|
||||||
this.resourceLimit = resourceLimit;
|
this.resourceLimit = resourceLimit;
|
||||||
this.strictContainers = strictContainers;
|
this.strictContainers = strictContainers;
|
||||||
@ -65,6 +77,8 @@ public Allocation(List<Container> containers, Resource resourceLimit,
|
|||||||
this.nmTokens = nmTokens;
|
this.nmTokens = nmTokens;
|
||||||
this.increasedContainers = increasedContainers;
|
this.increasedContainers = increasedContainers;
|
||||||
this.decreasedContainers = decreasedContainer;
|
this.decreasedContainers = decreasedContainer;
|
||||||
|
this.promotedContainers = promotedContainers;
|
||||||
|
this.demotedContainers = demotedContainer;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<Container> getContainers() {
|
public List<Container> getContainers() {
|
||||||
@ -99,6 +113,14 @@ public List<Container> getDecreasedContainers() {
|
|||||||
return decreasedContainers;
|
return decreasedContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<Container> getPromotedContainers() {
|
||||||
|
return promotedContainers;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<Container> getDemotedContainers() {
|
||||||
|
return demotedContainers;
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public void setResourceLimit(Resource resource) {
|
public void setResourceLimit(Resource resource) {
|
||||||
this.resourceLimit = resource;
|
this.resourceLimit = resource;
|
||||||
|
@ -55,7 +55,6 @@
|
|||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class keeps track of all the consumption of an application. This also
|
* This class keeps track of all the consumption of an application. This also
|
||||||
* keeps track of current running/completed containers for the application.
|
* keeps track of current running/completed containers for the application.
|
||||||
@ -92,10 +91,11 @@ public class AppSchedulingInfo {
|
|||||||
final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
|
final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
|
||||||
SchedContainerChangeRequest>>> containerIncreaseRequestMap =
|
SchedContainerChangeRequest>>> containerIncreaseRequestMap =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
|
|
||||||
private final ReentrantReadWriteLock.ReadLock readLock;
|
private final ReentrantReadWriteLock.ReadLock readLock;
|
||||||
private final ReentrantReadWriteLock.WriteLock writeLock;
|
private final ReentrantReadWriteLock.WriteLock writeLock;
|
||||||
|
|
||||||
|
public final ContainerUpdateContext updateContext;
|
||||||
|
|
||||||
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
||||||
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
||||||
long epoch, ResourceUsage appResourceUsage) {
|
long epoch, ResourceUsage appResourceUsage) {
|
||||||
@ -109,6 +109,7 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
|
|||||||
this.appResourceUsage = appResourceUsage;
|
this.appResourceUsage = appResourceUsage;
|
||||||
|
|
||||||
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
|
updateContext = new ContainerUpdateContext(this);
|
||||||
readLock = lock.readLock();
|
readLock = lock.readLock();
|
||||||
writeLock = lock.writeLock();
|
writeLock = lock.writeLock();
|
||||||
}
|
}
|
||||||
@ -376,6 +377,10 @@ public SchedContainerChangeRequest getIncreaseRequest(NodeId nodeId,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ContainerUpdateContext getUpdateContext() {
|
||||||
|
return updateContext;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The ApplicationMaster is updating resource requirements for the
|
* The ApplicationMaster is updating resource requirements for the
|
||||||
* application, by asking for more resources and releasing resources acquired
|
* application, by asking for more resources and releasing resources acquired
|
||||||
@ -413,29 +418,9 @@ public boolean updateResourceRequests(List<ResourceRequest> requests,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Update scheduling placement set
|
// Update scheduling placement set
|
||||||
for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry : dedupRequests.entrySet()) {
|
offswitchResourcesUpdated =
|
||||||
SchedulerRequestKey schedulerRequestKey = entry.getKey();
|
addToPlacementSets(
|
||||||
|
recoverPreemptedRequestForAContainer, dedupRequests);
|
||||||
if (!schedulerKeyToPlacementSets.containsKey(schedulerRequestKey)) {
|
|
||||||
schedulerKeyToPlacementSets.put(schedulerRequestKey,
|
|
||||||
new LocalitySchedulingPlacementSet<>(this));
|
|
||||||
}
|
|
||||||
|
|
||||||
// Update placement set
|
|
||||||
ResourceRequestUpdateResult pendingAmountChanges =
|
|
||||||
schedulerKeyToPlacementSets.get(schedulerRequestKey)
|
|
||||||
.updateResourceRequests(
|
|
||||||
entry.getValue().values(),
|
|
||||||
recoverPreemptedRequestForAContainer);
|
|
||||||
|
|
||||||
if (null != pendingAmountChanges) {
|
|
||||||
updatePendingResources(
|
|
||||||
pendingAmountChanges.getLastAnyResourceRequest(),
|
|
||||||
pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
|
|
||||||
queue.getMetrics());
|
|
||||||
offswitchResourcesUpdated = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return offswitchResourcesUpdated;
|
return offswitchResourcesUpdated;
|
||||||
} finally {
|
} finally {
|
||||||
@ -443,6 +428,37 @@ public boolean updateResourceRequests(List<ResourceRequest> requests,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
boolean addToPlacementSets(
|
||||||
|
boolean recoverPreemptedRequestForAContainer,
|
||||||
|
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
|
||||||
|
boolean offswitchResourcesUpdated = false;
|
||||||
|
for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
|
||||||
|
dedupRequests.entrySet()) {
|
||||||
|
SchedulerRequestKey schedulerRequestKey = entry.getKey();
|
||||||
|
|
||||||
|
if (!schedulerKeyToPlacementSets.containsKey(schedulerRequestKey)) {
|
||||||
|
schedulerKeyToPlacementSets.put(schedulerRequestKey,
|
||||||
|
new LocalitySchedulingPlacementSet<>(this));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Update placement set
|
||||||
|
ResourceRequestUpdateResult pendingAmountChanges =
|
||||||
|
schedulerKeyToPlacementSets.get(schedulerRequestKey)
|
||||||
|
.updateResourceRequests(
|
||||||
|
entry.getValue().values(),
|
||||||
|
recoverPreemptedRequestForAContainer);
|
||||||
|
|
||||||
|
if (null != pendingAmountChanges) {
|
||||||
|
updatePendingResources(
|
||||||
|
pendingAmountChanges.getLastAnyResourceRequest(),
|
||||||
|
pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
|
||||||
|
queue.getMetrics());
|
||||||
|
offswitchResourcesUpdated = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return offswitchResourcesUpdated;
|
||||||
|
}
|
||||||
|
|
||||||
private void updatePendingResources(ResourceRequest lastRequest,
|
private void updatePendingResources(ResourceRequest lastRequest,
|
||||||
ResourceRequest request, SchedulerRequestKey schedulerKey,
|
ResourceRequest request, SchedulerRequestKey schedulerKey,
|
||||||
QueueMetrics metrics) {
|
QueueMetrics metrics) {
|
||||||
@ -717,8 +733,8 @@ public List<ResourceRequest> allocate(NodeType type,
|
|||||||
updateMetricsForAllocatedContainer(type, containerAllocated);
|
updateMetricsForAllocatedContainer(type, containerAllocated);
|
||||||
}
|
}
|
||||||
|
|
||||||
return schedulerKeyToPlacementSets.get(schedulerKey).allocate(type, node,
|
return schedulerKeyToPlacementSets.get(schedulerKey)
|
||||||
request);
|
.allocate(schedulerKey, type, node, request);
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,267 @@
|
|||||||
|
/**
|
||||||
|
* 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.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.ContainerUpdateType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
||||||
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class encapsulates all outstanding container increase and decrease
|
||||||
|
* requests for an application.
|
||||||
|
*/
|
||||||
|
public class ContainerUpdateContext {
|
||||||
|
|
||||||
|
public static final ContainerId UNDEFINED =
|
||||||
|
ContainerId.newContainerId(ApplicationAttemptId.newInstance(
|
||||||
|
ApplicationId.newInstance(-1, -1), -1), -1);
|
||||||
|
protected static final RecordFactory RECORD_FACTORY =
|
||||||
|
RecordFactoryProvider.getRecordFactory(null);
|
||||||
|
|
||||||
|
// Keep track of containers that are undergoing promotion
|
||||||
|
private final Map<SchedulerRequestKey, Map<Resource,
|
||||||
|
Map<NodeId, Set<ContainerId>>>> outstandingIncreases = new HashMap<>();
|
||||||
|
|
||||||
|
private final Set<ContainerId> outstandingDecreases = new HashSet<>();
|
||||||
|
private final AppSchedulingInfo appSchedulingInfo;
|
||||||
|
|
||||||
|
ContainerUpdateContext(AppSchedulingInfo appSchedulingInfo) {
|
||||||
|
this.appSchedulingInfo = appSchedulingInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized boolean isBeingIncreased(Container container) {
|
||||||
|
Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
|
||||||
|
outstandingIncreases.get(
|
||||||
|
new SchedulerRequestKey(container.getPriority(),
|
||||||
|
container.getAllocationRequestId(), container.getId()));
|
||||||
|
if (resourceMap != null) {
|
||||||
|
Map<NodeId, Set<ContainerId>> locationMap =
|
||||||
|
resourceMap.get(container.getResource());
|
||||||
|
if (locationMap != null) {
|
||||||
|
Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
|
||||||
|
if (containerIds != null && !containerIds.isEmpty()) {
|
||||||
|
return containerIds.contains(container.getId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the container to outstanding decreases.
|
||||||
|
* @param container Container.
|
||||||
|
* @return true if updated to outstanding decreases was successful.
|
||||||
|
*/
|
||||||
|
public synchronized boolean checkAndAddToOutstandingDecreases(
|
||||||
|
Container container) {
|
||||||
|
if (isBeingIncreased(container)
|
||||||
|
|| outstandingDecreases.contains(container.getId())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
outstandingDecreases.add(container.getId());
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the container to outstanding increases.
|
||||||
|
* @param rmContainer RMContainer.
|
||||||
|
* @param schedulerNode SchedulerNode.
|
||||||
|
* @param updateRequest UpdateContainerRequest.
|
||||||
|
* @return true if updated to outstanding increases was successful.
|
||||||
|
*/
|
||||||
|
public synchronized boolean checkAndAddToOutstandingIncreases(
|
||||||
|
RMContainer rmContainer, SchedulerNode schedulerNode,
|
||||||
|
UpdateContainerRequest updateRequest) {
|
||||||
|
Container container = rmContainer.getContainer();
|
||||||
|
SchedulerRequestKey schedulerKey =
|
||||||
|
SchedulerRequestKey.create(updateRequest,
|
||||||
|
rmContainer.getAllocatedSchedulerKey());
|
||||||
|
Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
|
||||||
|
outstandingIncreases.get(schedulerKey);
|
||||||
|
if (resourceMap == null) {
|
||||||
|
resourceMap = new HashMap<>();
|
||||||
|
outstandingIncreases.put(schedulerKey, resourceMap);
|
||||||
|
}
|
||||||
|
Map<NodeId, Set<ContainerId>> locationMap =
|
||||||
|
resourceMap.get(container.getResource());
|
||||||
|
if (locationMap == null) {
|
||||||
|
locationMap = new HashMap<>();
|
||||||
|
resourceMap.put(container.getResource(), locationMap);
|
||||||
|
}
|
||||||
|
Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
|
||||||
|
if (containerIds == null) {
|
||||||
|
containerIds = new HashSet<>();
|
||||||
|
locationMap.put(container.getNodeId(), containerIds);
|
||||||
|
}
|
||||||
|
if (containerIds.contains(container.getId())
|
||||||
|
|| outstandingDecreases.contains(container.getId())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
containerIds.add(container.getId());
|
||||||
|
|
||||||
|
Map<SchedulerRequestKey, Map<String, ResourceRequest>> updateResReqs =
|
||||||
|
new HashMap<>();
|
||||||
|
Resource resToIncrease = getResourceToIncrease(updateRequest, rmContainer);
|
||||||
|
Map<String, ResourceRequest> resMap =
|
||||||
|
createResourceRequests(rmContainer, schedulerNode,
|
||||||
|
schedulerKey, resToIncrease);
|
||||||
|
updateResReqs.put(schedulerKey, resMap);
|
||||||
|
appSchedulingInfo.addToPlacementSets(false, updateResReqs);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, ResourceRequest> createResourceRequests(
|
||||||
|
RMContainer rmContainer, SchedulerNode schedulerNode,
|
||||||
|
SchedulerRequestKey schedulerKey, Resource resToIncrease) {
|
||||||
|
Map<String, ResourceRequest> resMap = new HashMap<>();
|
||||||
|
resMap.put(rmContainer.getContainer().getNodeId().getHost(),
|
||||||
|
createResourceReqForIncrease(schedulerKey, resToIncrease,
|
||||||
|
RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
|
||||||
|
rmContainer, rmContainer.getContainer().getNodeId().getHost()));
|
||||||
|
resMap.put(schedulerNode.getRackName(),
|
||||||
|
createResourceReqForIncrease(schedulerKey, resToIncrease,
|
||||||
|
RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
|
||||||
|
rmContainer, schedulerNode.getRackName()));
|
||||||
|
resMap.put(ResourceRequest.ANY,
|
||||||
|
createResourceReqForIncrease(schedulerKey, resToIncrease,
|
||||||
|
RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
|
||||||
|
rmContainer, ResourceRequest.ANY));
|
||||||
|
return resMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Resource getResourceToIncrease(UpdateContainerRequest updateReq,
|
||||||
|
RMContainer rmContainer) {
|
||||||
|
if (updateReq.getContainerUpdateType() ==
|
||||||
|
ContainerUpdateType.PROMOTE_EXECUTION_TYPE) {
|
||||||
|
return rmContainer.getContainer().getResource();
|
||||||
|
}
|
||||||
|
// TODO: Fix this for container increase..
|
||||||
|
// This has to equal the Resources in excess of fitsIn()
|
||||||
|
// for container increase and is equal to the container total
|
||||||
|
// resource for Promotion.
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static ResourceRequest createResourceReqForIncrease(
|
||||||
|
SchedulerRequestKey schedulerRequestKey, Resource resToIncrease,
|
||||||
|
ResourceRequest rr, RMContainer rmContainer, String resourceName) {
|
||||||
|
rr.setResourceName(resourceName);
|
||||||
|
rr.setNumContainers(1);
|
||||||
|
rr.setRelaxLocality(false);
|
||||||
|
rr.setPriority(rmContainer.getContainer().getPriority());
|
||||||
|
rr.setAllocationRequestId(schedulerRequestKey.getAllocationRequestId());
|
||||||
|
rr.setCapability(resToIncrease);
|
||||||
|
rr.setNodeLabelExpression(rmContainer.getNodeLabelExpression());
|
||||||
|
rr.setExecutionTypeRequest(ExecutionTypeRequest.newInstance(
|
||||||
|
ExecutionType.GUARANTEED, true));
|
||||||
|
return rr;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove Container from outstanding increases / decreases. Calling this
|
||||||
|
* method essentially completes the update process.
|
||||||
|
* @param schedulerKey SchedulerRequestKey.
|
||||||
|
* @param container Container.
|
||||||
|
*/
|
||||||
|
public synchronized void removeFromOutstandingUpdate(
|
||||||
|
SchedulerRequestKey schedulerKey, Container container) {
|
||||||
|
Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
|
||||||
|
outstandingIncreases.get(schedulerKey);
|
||||||
|
if (resourceMap != null) {
|
||||||
|
Map<NodeId, Set<ContainerId>> locationMap =
|
||||||
|
resourceMap.get(container.getResource());
|
||||||
|
if (locationMap != null) {
|
||||||
|
Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
|
||||||
|
if (containerIds != null && !containerIds.isEmpty()) {
|
||||||
|
containerIds.remove(container.getId());
|
||||||
|
if (containerIds.isEmpty()) {
|
||||||
|
locationMap.remove(container.getNodeId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (locationMap.isEmpty()) {
|
||||||
|
resourceMap.remove(container.getResource());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (resourceMap.isEmpty()) {
|
||||||
|
outstandingIncreases.remove(schedulerKey);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
outstandingDecreases.remove(container.getId());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if a new container is to be matched up against an outstanding
|
||||||
|
* Container increase request.
|
||||||
|
* @param schedulerKey SchedulerRequestKey.
|
||||||
|
* @param rmContainer RMContainer.
|
||||||
|
* @return ContainerId.
|
||||||
|
*/
|
||||||
|
public ContainerId matchContainerToOutstandingIncreaseReq(
|
||||||
|
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||||
|
RMContainer rmContainer) {
|
||||||
|
ContainerId retVal = null;
|
||||||
|
Container container = rmContainer.getContainer();
|
||||||
|
Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
|
||||||
|
outstandingIncreases.get(schedulerKey);
|
||||||
|
if (resourceMap != null) {
|
||||||
|
Map<NodeId, Set<ContainerId>> locationMap =
|
||||||
|
resourceMap.get(container.getResource());
|
||||||
|
if (locationMap != null) {
|
||||||
|
Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
|
||||||
|
if (containerIds != null && !containerIds.isEmpty()) {
|
||||||
|
retVal = containerIds.iterator().next();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Allocation happened on NM on the same host, but not on the NM
|
||||||
|
// we need.. We need to signal that this container has to be released.
|
||||||
|
// We also need to add these requests back.. to be reallocated.
|
||||||
|
if (resourceMap != null && retVal == null) {
|
||||||
|
Map<SchedulerRequestKey, Map<String, ResourceRequest>> reqsToUpdate =
|
||||||
|
new HashMap<>();
|
||||||
|
Map<String, ResourceRequest> resMap = createResourceRequests
|
||||||
|
(rmContainer, node, schedulerKey,
|
||||||
|
rmContainer.getContainer().getResource());
|
||||||
|
reqsToUpdate.put(schedulerKey, resMap);
|
||||||
|
appSchedulingInfo.addToPlacementSets(true, reqsToUpdate);
|
||||||
|
return UNDEFINED;
|
||||||
|
}
|
||||||
|
return retVal;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,68 @@
|
|||||||
|
/**
|
||||||
|
* 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.UpdateContainerRequest;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holder class that maintains list of container update requests
|
||||||
|
*/
|
||||||
|
public class ContainerUpdates {
|
||||||
|
|
||||||
|
final List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
|
||||||
|
final List<UpdateContainerRequest> decreaseRequests = new ArrayList<>();
|
||||||
|
final List<UpdateContainerRequest> promotionRequests = new ArrayList<>();
|
||||||
|
final List<UpdateContainerRequest> demotionRequests = new ArrayList<>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Container Increase Requests.
|
||||||
|
* @return Container Increase Requests.
|
||||||
|
*/
|
||||||
|
public List<UpdateContainerRequest> getIncreaseRequests() {
|
||||||
|
return increaseRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Container Decrease Requests.
|
||||||
|
* @return Container Decrease Requests.
|
||||||
|
*/
|
||||||
|
public List<UpdateContainerRequest> getDecreaseRequests() {
|
||||||
|
return decreaseRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Container Promotion Requests.
|
||||||
|
* @return Container Promotion Requests.
|
||||||
|
*/
|
||||||
|
public List<UpdateContainerRequest> getPromotionRequests() {
|
||||||
|
return promotionRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns Container Demotion Requests.
|
||||||
|
* @return Container Demotion Requests.
|
||||||
|
*/
|
||||||
|
public List<UpdateContainerRequest> getDemotionRequests() {
|
||||||
|
return demotionRequests;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -18,6 +18,7 @@
|
|||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
@ -47,6 +48,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
|
||||||
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||||
import org.apache.hadoop.yarn.api.records.LogAggregationContext;
|
import org.apache.hadoop.yarn.api.records.LogAggregationContext;
|
||||||
import org.apache.hadoop.yarn.api.records.NMToken;
|
import org.apache.hadoop.yarn.api.records.NMToken;
|
||||||
@ -54,6 +56,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
|
||||||
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
||||||
import org.apache.hadoop.yarn.server.api.ContainerType;
|
import org.apache.hadoop.yarn.server.api.ContainerType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||||
@ -133,10 +136,15 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||||||
private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
|
private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
|
||||||
|
|
||||||
protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
|
protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
|
||||||
|
protected Map<ContainerId, RMContainer> newlyPromotedContainers = new HashMap<>();
|
||||||
|
protected Map<ContainerId, RMContainer> newlyDemotedContainers = new HashMap<>();
|
||||||
|
protected List<RMContainer> tempContainerToKill = new ArrayList<>();
|
||||||
protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
|
protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
|
||||||
protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
|
protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
|
||||||
protected Set<NMToken> updatedNMTokens = new HashSet<>();
|
protected Set<NMToken> updatedNMTokens = new HashSet<>();
|
||||||
|
|
||||||
|
protected List<UpdateContainerError> updateContainerErrors = new ArrayList<>();
|
||||||
|
|
||||||
// This pendingRelease is used in work-preserving recovery scenario to keep
|
// This pendingRelease is used in work-preserving recovery scenario to keep
|
||||||
// track of the AM's outstanding release requests. RM on recovery could
|
// track of the AM's outstanding release requests. RM on recovery could
|
||||||
// receive the release request form AM before it receives the container status
|
// receive the release request form AM before it receives the container status
|
||||||
@ -247,6 +255,10 @@ public AppSchedulingInfo getAppSchedulingInfo() {
|
|||||||
return this.appSchedulingInfo;
|
return this.appSchedulingInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ContainerUpdateContext getUpdateContext() {
|
||||||
|
return this.appSchedulingInfo.getUpdateContext();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Is this application pending?
|
* Is this application pending?
|
||||||
* @return true if it is else false.
|
* @return true if it is else false.
|
||||||
@ -537,8 +549,9 @@ public RMContainer reserve(SchedulerNode node,
|
|||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
// Create RMContainer if necessary
|
// Create RMContainer if necessary
|
||||||
if (rmContainer == null) {
|
if (rmContainer == null) {
|
||||||
rmContainer = new RMContainerImpl(container, getApplicationAttemptId(),
|
rmContainer = new RMContainerImpl(container, schedulerKey,
|
||||||
node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
|
getApplicationAttemptId(), node.getNodeID(),
|
||||||
|
appSchedulingInfo.getUser(), rmContext);
|
||||||
}
|
}
|
||||||
if (rmContainer.getState() == RMContainerState.NEW) {
|
if (rmContainer.getState() == RMContainerState.NEW) {
|
||||||
attemptResourceUsage.incReserved(node.getPartition(),
|
attemptResourceUsage.incReserved(node.getPartition(),
|
||||||
@ -635,10 +648,10 @@ public Resource getCurrentConsumption() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private Container updateContainerAndNMToken(RMContainer rmContainer,
|
private Container updateContainerAndNMToken(RMContainer rmContainer,
|
||||||
boolean newContainer, boolean increasedContainer) {
|
ContainerUpdateType updateType) {
|
||||||
Container container = rmContainer.getContainer();
|
Container container = rmContainer.getContainer();
|
||||||
ContainerType containerType = ContainerType.TASK;
|
ContainerType containerType = ContainerType.TASK;
|
||||||
if (!newContainer) {
|
if (updateType != null) {
|
||||||
container.setVersion(container.getVersion() + 1);
|
container.setVersion(container.getVersion() + 1);
|
||||||
}
|
}
|
||||||
// The working knowledge is that masterContainer for AM is null as it
|
// The working knowledge is that masterContainer for AM is null as it
|
||||||
@ -662,12 +675,15 @@ private Container updateContainerAndNMToken(RMContainer rmContainer,
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (newContainer) {
|
if (updateType == null ||
|
||||||
|
ContainerUpdateType.PROMOTE_EXECUTION_TYPE == updateType ||
|
||||||
|
ContainerUpdateType.DEMOTE_EXECUTION_TYPE == updateType) {
|
||||||
rmContainer.handle(new RMContainerEvent(
|
rmContainer.handle(new RMContainerEvent(
|
||||||
rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
|
rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
|
||||||
} else {
|
} else {
|
||||||
rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
|
rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
|
||||||
rmContainer.getContainerId(), increasedContainer));
|
rmContainer.getContainerId(),
|
||||||
|
ContainerUpdateType.INCREASE_RESOURCE == updateType));
|
||||||
}
|
}
|
||||||
return container;
|
return container;
|
||||||
}
|
}
|
||||||
@ -699,8 +715,8 @@ public List<Container> pullNewlyAllocatedContainers() {
|
|||||||
Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
|
Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
|
||||||
while (i.hasNext()) {
|
while (i.hasNext()) {
|
||||||
RMContainer rmContainer = i.next();
|
RMContainer rmContainer = i.next();
|
||||||
Container updatedContainer = updateContainerAndNMToken(rmContainer,
|
Container updatedContainer =
|
||||||
true, false);
|
updateContainerAndNMToken(rmContainer, null);
|
||||||
// Only add container to return list when it's not null.
|
// Only add container to return list when it's not null.
|
||||||
// updatedContainer could be null when generate token failed, it can be
|
// updatedContainer could be null when generate token failed, it can be
|
||||||
// caused by DNS resolving failed.
|
// caused by DNS resolving failed.
|
||||||
@ -713,9 +729,142 @@ public List<Container> pullNewlyAllocatedContainers() {
|
|||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void addToNewlyDemotedContainers(ContainerId containerId,
|
||||||
|
RMContainer rmContainer) {
|
||||||
|
newlyDemotedContainers.put(containerId, rmContainer);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected synchronized void addToUpdateContainerErrors(
|
||||||
|
UpdateContainerError error) {
|
||||||
|
updateContainerErrors.add(error);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected synchronized void addToNewlyAllocatedContainers(
|
||||||
|
SchedulerNode node, RMContainer rmContainer) {
|
||||||
|
if (oppContainerContext == null) {
|
||||||
|
newlyAllocatedContainers.add(rmContainer);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
ContainerId matchedContainerId =
|
||||||
|
getUpdateContext().matchContainerToOutstandingIncreaseReq(
|
||||||
|
node, rmContainer.getAllocatedSchedulerKey(), rmContainer);
|
||||||
|
if (matchedContainerId != null) {
|
||||||
|
if (ContainerUpdateContext.UNDEFINED == matchedContainerId) {
|
||||||
|
// This is a spurious allocation (relaxLocality = false
|
||||||
|
// resulted in the Container being allocated on an NM on the same host
|
||||||
|
// but not on the NM running the container to be updated. Can
|
||||||
|
// happen if more than one NM exists on the same host.. usually
|
||||||
|
// occurs when using MiniYARNCluster to test).
|
||||||
|
tempContainerToKill.add(rmContainer);
|
||||||
|
} else {
|
||||||
|
newlyPromotedContainers.put(matchedContainerId, rmContainer);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
newlyAllocatedContainers.add(rmContainer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<Container> pullNewlyPromotedContainers() {
|
||||||
|
return pullContainersWithUpdatedExecType(newlyPromotedContainers,
|
||||||
|
ContainerUpdateType.PROMOTE_EXECUTION_TYPE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<Container> pullNewlyDemotedContainers() {
|
||||||
|
return pullContainersWithUpdatedExecType(newlyDemotedContainers,
|
||||||
|
ContainerUpdateType.DEMOTE_EXECUTION_TYPE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<UpdateContainerError> pullUpdateContainerErrors() {
|
||||||
|
List<UpdateContainerError> errors =
|
||||||
|
new ArrayList<>(updateContainerErrors);
|
||||||
|
updateContainerErrors.clear();
|
||||||
|
return errors;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A container is promoted if its executionType is changed from
|
||||||
|
* OPPORTUNISTIC to GUARANTEED. It id demoted if the change is from
|
||||||
|
* GUARANTEED to OPPORTUNISTIC.
|
||||||
|
* @return Newly Promoted and Demoted containers
|
||||||
|
*/
|
||||||
|
private List<Container> pullContainersWithUpdatedExecType(
|
||||||
|
Map<ContainerId, RMContainer> newlyUpdatedContainers,
|
||||||
|
ContainerUpdateType updateTpe) {
|
||||||
|
List<Container> updatedContainers = new ArrayList<>();
|
||||||
|
if (oppContainerContext == null) {
|
||||||
|
return updatedContainers;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
writeLock.lock();
|
||||||
|
Iterator<Map.Entry<ContainerId, RMContainer>> i =
|
||||||
|
newlyUpdatedContainers.entrySet().iterator();
|
||||||
|
while (i.hasNext()) {
|
||||||
|
Map.Entry<ContainerId, RMContainer> entry = i.next();
|
||||||
|
ContainerId matchedContainerId = entry.getKey();
|
||||||
|
RMContainer rmContainer = entry.getValue();
|
||||||
|
|
||||||
|
// swap containers
|
||||||
|
RMContainer existingRMContainer = swapContainer(
|
||||||
|
rmContainer, matchedContainerId);
|
||||||
|
getUpdateContext().removeFromOutstandingUpdate(
|
||||||
|
rmContainer.getAllocatedSchedulerKey(),
|
||||||
|
existingRMContainer.getContainer());
|
||||||
|
Container updatedContainer = updateContainerAndNMToken(
|
||||||
|
existingRMContainer, updateTpe);
|
||||||
|
updatedContainers.add(updatedContainer);
|
||||||
|
|
||||||
|
tempContainerToKill.add(rmContainer);
|
||||||
|
i.remove();
|
||||||
|
}
|
||||||
|
// Release all temporary containers
|
||||||
|
Iterator<RMContainer> tempIter = tempContainerToKill.iterator();
|
||||||
|
while (tempIter.hasNext()) {
|
||||||
|
RMContainer c = tempIter.next();
|
||||||
|
// Mark container for release (set RRs to null, so RM does not think
|
||||||
|
// it is a recoverable container)
|
||||||
|
((RMContainerImpl) c).setResourceRequests(null);
|
||||||
|
((AbstractYarnScheduler) rmContext.getScheduler()).completedContainer(c,
|
||||||
|
SchedulerUtils.createAbnormalContainerStatus(c.getContainerId(),
|
||||||
|
SchedulerUtils.UPDATED_CONTAINER),
|
||||||
|
RMContainerEventType.KILL);
|
||||||
|
tempIter.remove();
|
||||||
|
}
|
||||||
|
return updatedContainers;
|
||||||
|
} finally {
|
||||||
|
writeLock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private RMContainer swapContainer(RMContainer rmContainer, ContainerId
|
||||||
|
matchedContainerId) {
|
||||||
|
RMContainer existingRMContainer =
|
||||||
|
getRMContainer(matchedContainerId);
|
||||||
|
if (existingRMContainer != null) {
|
||||||
|
// Swap updated container with the existing container
|
||||||
|
Container updatedContainer = rmContainer.getContainer();
|
||||||
|
|
||||||
|
Container newContainer = Container.newInstance(matchedContainerId,
|
||||||
|
existingRMContainer.getContainer().getNodeId(),
|
||||||
|
existingRMContainer.getContainer().getNodeHttpAddress(),
|
||||||
|
updatedContainer.getResource(),
|
||||||
|
existingRMContainer.getContainer().getPriority(), null,
|
||||||
|
updatedContainer.getExecutionType());
|
||||||
|
newContainer.setAllocationRequestId(
|
||||||
|
existingRMContainer.getContainer().getAllocationRequestId());
|
||||||
|
newContainer.setVersion(existingRMContainer.getContainer().getVersion());
|
||||||
|
|
||||||
|
rmContainer.getContainer().setResource(
|
||||||
|
existingRMContainer.getContainer().getResource());
|
||||||
|
rmContainer.getContainer().setExecutionType(
|
||||||
|
existingRMContainer.getContainer().getExecutionType());
|
||||||
|
|
||||||
|
((RMContainerImpl)existingRMContainer).setContainer(newContainer);
|
||||||
|
}
|
||||||
|
return existingRMContainer;
|
||||||
|
}
|
||||||
|
|
||||||
private List<Container> pullNewlyUpdatedContainers(
|
private List<Container> pullNewlyUpdatedContainers(
|
||||||
Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
|
Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
|
||||||
try {
|
try {
|
||||||
@ -728,7 +877,8 @@ private List<Container> pullNewlyUpdatedContainers(
|
|||||||
while (i.hasNext()) {
|
while (i.hasNext()) {
|
||||||
RMContainer rmContainer = i.next().getValue();
|
RMContainer rmContainer = i.next().getValue();
|
||||||
Container updatedContainer = updateContainerAndNMToken(rmContainer,
|
Container updatedContainer = updateContainerAndNMToken(rmContainer,
|
||||||
false, increase);
|
increase ? ContainerUpdateType.INCREASE_RESOURCE :
|
||||||
|
ContainerUpdateType.DECREASE_RESOURCE);
|
||||||
if (updatedContainer != null) {
|
if (updatedContainer != null) {
|
||||||
returnContainerList.add(updatedContainer);
|
returnContainerList.add(updatedContainer);
|
||||||
i.remove();
|
i.remove();
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
@ -41,7 +42,10 @@
|
|||||||
import org.apache.hadoop.yarn.security.AccessType;
|
import org.apache.hadoop.yarn.security.AccessType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
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.RMContainerImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
@ -57,6 +61,9 @@ public class SchedulerUtils {
|
|||||||
|
|
||||||
public static final String RELEASED_CONTAINER =
|
public static final String RELEASED_CONTAINER =
|
||||||
"Container released by application";
|
"Container released by application";
|
||||||
|
|
||||||
|
public static final String UPDATED_CONTAINER =
|
||||||
|
"Temporary container killed by application for ExeutionType update";
|
||||||
|
|
||||||
public static final String LOST_CONTAINER =
|
public static final String LOST_CONTAINER =
|
||||||
"Container released on a *lost* node";
|
"Container released on a *lost* node";
|
||||||
@ -376,4 +383,19 @@ public static boolean hasPendingResourceRequest(ResourceCalculator rc,
|
|||||||
}
|
}
|
||||||
return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
|
return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static RMContainer createOpportunisticRmContainer(RMContext rmContext,
|
||||||
|
Container container, boolean isRemotelyAllocated) {
|
||||||
|
SchedulerApplicationAttempt appAttempt =
|
||||||
|
((AbstractYarnScheduler) rmContext.getScheduler())
|
||||||
|
.getCurrentAttemptForContainer(container.getId());
|
||||||
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container),
|
||||||
|
appAttempt.getApplicationAttemptId(), container.getNodeId(),
|
||||||
|
appAttempt.getUser(), rmContext, isRemotelyAllocated);
|
||||||
|
appAttempt.addRMContainer(container.getId(), rmContainer);
|
||||||
|
((AbstractYarnScheduler) rmContext.getScheduler()).getNode(
|
||||||
|
container.getNodeId()).allocateContainer(rmContainer);
|
||||||
|
return rmContainer;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -30,6 +30,7 @@
|
|||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.yarn.api.records.AbstractResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
||||||
@ -42,8 +43,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.records.AbstractResourceRequest;
|
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
@ -137,8 +136,7 @@ public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
|
|||||||
* @param release
|
* @param release
|
||||||
* @param blacklistAdditions
|
* @param blacklistAdditions
|
||||||
* @param blacklistRemovals
|
* @param blacklistRemovals
|
||||||
* @param increaseRequests
|
* @param updateRequests
|
||||||
* @param decreaseRequests
|
|
||||||
* @return the {@link Allocation} for the application
|
* @return the {@link Allocation} for the application
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@ -146,8 +144,7 @@ public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
|
|||||||
Allocation allocate(ApplicationAttemptId appAttemptId,
|
Allocation allocate(ApplicationAttemptId appAttemptId,
|
||||||
List<ResourceRequest> ask, List<ContainerId> release,
|
List<ResourceRequest> ask, List<ContainerId> release,
|
||||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests);
|
||||||
List<UpdateContainerRequest> decreaseRequests);
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get node resource usage report.
|
* Get node resource usage report.
|
||||||
|
@ -90,6 +90,9 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||||
@ -921,22 +924,27 @@ private LeafQueue updateIncreaseRequests(
|
|||||||
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
|
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
|
||||||
List<ResourceRequest> ask, List<ContainerId> release,
|
List<ResourceRequest> ask, List<ContainerId> release,
|
||||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
return EMPTY_ALLOCATION;
|
return EMPTY_ALLOCATION;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Handle promotions and demotions
|
||||||
|
handleExecutionTypeUpdates(
|
||||||
|
application, updateRequests.getPromotionRequests(),
|
||||||
|
updateRequests.getDemotionRequests());
|
||||||
|
|
||||||
// Release containers
|
// Release containers
|
||||||
releaseContainers(release, application);
|
releaseContainers(release, application);
|
||||||
|
|
||||||
// update increase requests
|
// update increase requests
|
||||||
LeafQueue updateDemandForQueue = updateIncreaseRequests(increaseRequests,
|
LeafQueue updateDemandForQueue =
|
||||||
|
updateIncreaseRequests(updateRequests.getIncreaseRequests(),
|
||||||
application);
|
application);
|
||||||
|
|
||||||
// Decrease containers
|
// Decrease containers
|
||||||
decreaseContainers(decreaseRequests, application);
|
decreaseContainers(updateRequests.getDecreaseRequests(), application);
|
||||||
|
|
||||||
// Sanity check for new allocation requests
|
// Sanity check for new allocation requests
|
||||||
normalizeRequests(ask);
|
normalizeRequests(ask);
|
||||||
|
@ -746,7 +746,7 @@ ContainerAllocation doAllocation(ContainerAllocation allocationResult,
|
|||||||
// When reserving container
|
// When reserving container
|
||||||
RMContainer updatedContainer = reservedContainer;
|
RMContainer updatedContainer = reservedContainer;
|
||||||
if (updatedContainer == null) {
|
if (updatedContainer == null) {
|
||||||
updatedContainer = new RMContainerImpl(container,
|
updatedContainer = new RMContainerImpl(container, schedulerKey,
|
||||||
application.getApplicationAttemptId(), node.getNodeID(),
|
application.getApplicationAttemptId(), node.getNodeID(),
|
||||||
application.getAppSchedulingInfo().getUser(), rmContext);
|
application.getAppSchedulingInfo().getUser(), rmContext);
|
||||||
}
|
}
|
||||||
|
@ -222,7 +222,7 @@ public RMContainer allocate(FiCaSchedulerNode node,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Create RMContainer
|
// Create RMContainer
|
||||||
RMContainer rmContainer = new RMContainerImpl(container,
|
RMContainer rmContainer = new RMContainerImpl(container, schedulerKey,
|
||||||
this.getApplicationAttemptId(), node.getNodeID(),
|
this.getApplicationAttemptId(), node.getNodeID(),
|
||||||
appSchedulingInfo.getUser(), this.rmContext,
|
appSchedulingInfo.getUser(), this.rmContext,
|
||||||
request.getNodeLabelExpression());
|
request.getNodeLabelExpression());
|
||||||
@ -554,12 +554,14 @@ public void apply(Resource cluster,
|
|||||||
// Update this application for the allocated container
|
// Update this application for the allocated container
|
||||||
if (!allocation.isIncreasedAllocation()) {
|
if (!allocation.isIncreasedAllocation()) {
|
||||||
// Allocate a new container
|
// Allocate a new container
|
||||||
newlyAllocatedContainers.add(rmContainer);
|
addToNewlyAllocatedContainers(
|
||||||
|
schedulerContainer.getSchedulerNode(), rmContainer);
|
||||||
liveContainers.put(containerId, rmContainer);
|
liveContainers.put(containerId, rmContainer);
|
||||||
|
|
||||||
// Deduct pending resource requests
|
// Deduct pending resource requests
|
||||||
List<ResourceRequest> requests = appSchedulingInfo.allocate(
|
List<ResourceRequest> requests = appSchedulingInfo.allocate(
|
||||||
allocation.getAllocationLocalityType(), schedulerContainer.getSchedulerNode(),
|
allocation.getAllocationLocalityType(),
|
||||||
|
schedulerContainer.getSchedulerNode(),
|
||||||
schedulerContainer.getSchedulerRequestKey(),
|
schedulerContainer.getSchedulerRequestKey(),
|
||||||
schedulerContainer.getRmContainer().getContainer());
|
schedulerContainer.getRmContainer().getContainer());
|
||||||
((RMContainerImpl) rmContainer).setResourceRequests(requests);
|
((RMContainerImpl) rmContainer).setResourceRequests(requests);
|
||||||
@ -751,12 +753,15 @@ public Allocation getAllocation(ResourceCalculator resourceCalculator,
|
|||||||
List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
|
List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
|
||||||
List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
|
List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
|
||||||
List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
|
List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
|
||||||
|
List<Container> newlyPromotedContainers = pullNewlyPromotedContainers();
|
||||||
|
List<Container> newlyDemotedContainers = pullNewlyDemotedContainers();
|
||||||
List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
|
List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
|
||||||
Resource headroom = getHeadroom();
|
Resource headroom = getHeadroom();
|
||||||
setApplicationHeadroomForMetrics(headroom);
|
setApplicationHeadroomForMetrics(headroom);
|
||||||
return new Allocation(newlyAllocatedContainers, headroom, null,
|
return new Allocation(newlyAllocatedContainers, headroom, null,
|
||||||
currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
|
currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
|
||||||
newlyIncreasedContainers, newlyDecreasedContainers);
|
newlyIncreasedContainers, newlyDecreasedContainers,
|
||||||
|
newlyPromotedContainers, newlyDemotedContainers);
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
|
@ -448,13 +448,13 @@ public RMContainer allocate(NodeType type, FSSchedulerNode node,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Create RMContainer
|
// Create RMContainer
|
||||||
rmContainer = new RMContainerImpl(container,
|
rmContainer = new RMContainerImpl(container, schedulerKey,
|
||||||
getApplicationAttemptId(), node.getNodeID(),
|
getApplicationAttemptId(), node.getNodeID(),
|
||||||
appSchedulingInfo.getUser(), rmContext);
|
appSchedulingInfo.getUser(), rmContext);
|
||||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||||
|
|
||||||
// Add it to allContainers list.
|
// Add it to allContainers list.
|
||||||
newlyAllocatedContainers.add(rmContainer);
|
addToNewlyAllocatedContainers(node, rmContainer);
|
||||||
liveContainers.put(container.getId(), rmContainer);
|
liveContainers.put(container.getId(), rmContainer);
|
||||||
|
|
||||||
// Update consumption and track allocations
|
// Update consumption and track allocations
|
||||||
|
@ -18,18 +18,8 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
import java.io.IOException;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import java.util.ArrayList;
|
import com.google.common.base.Preconditions;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.EnumSet;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
||||||
@ -51,7 +41,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
import org.apache.hadoop.yarn.api.records.ResourceOption;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
@ -80,6 +69,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
|
||||||
@ -102,8 +92,17 @@
|
|||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import java.io.IOException;
|
||||||
import com.google.common.base.Preconditions;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A scheduler that schedules resources between a set of queues. The scheduler
|
* A scheduler that schedules resources between a set of queues. The scheduler
|
||||||
@ -812,8 +811,7 @@ public void normalizeRequest(AbstractResourceRequest ask) {
|
|||||||
public Allocation allocate(ApplicationAttemptId appAttemptId,
|
public Allocation allocate(ApplicationAttemptId appAttemptId,
|
||||||
List<ResourceRequest> ask, List<ContainerId> release,
|
List<ResourceRequest> ask, List<ContainerId> release,
|
||||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
|
|
||||||
// Make sure this application exists
|
// Make sure this application exists
|
||||||
FSAppAttempt application = getSchedulerApp(appAttemptId);
|
FSAppAttempt application = getSchedulerApp(appAttemptId);
|
||||||
@ -823,6 +821,11 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
|
|||||||
return EMPTY_ALLOCATION;
|
return EMPTY_ALLOCATION;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Handle promotions and demotions
|
||||||
|
handleExecutionTypeUpdates(
|
||||||
|
application, updateRequests.getPromotionRequests(),
|
||||||
|
updateRequests.getDemotionRequests());
|
||||||
|
|
||||||
// Sanity check
|
// Sanity check
|
||||||
normalizeRequests(ask);
|
normalizeRequests(ask);
|
||||||
|
|
||||||
@ -879,7 +882,9 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
|
|||||||
application.setApplicationHeadroomForMetrics(headroom);
|
application.setApplicationHeadroomForMetrics(headroom);
|
||||||
return new Allocation(newlyAllocatedContainers, headroom,
|
return new Allocation(newlyAllocatedContainers, headroom,
|
||||||
preemptionContainerIds, null, null,
|
preemptionContainerIds, null, null,
|
||||||
application.pullUpdatedNMTokens());
|
application.pullUpdatedNMTokens(), null, null,
|
||||||
|
application.pullNewlyPromotedContainers(),
|
||||||
|
application.pullNewlyDemotedContainers());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -68,7 +68,7 @@ public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
|||||||
|
|
||||||
// Create RMContainer
|
// Create RMContainer
|
||||||
RMContainer rmContainer = new RMContainerImpl(container,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
this.getApplicationAttemptId(), node.getNodeID(),
|
schedulerKey, this.getApplicationAttemptId(), node.getNodeID(),
|
||||||
appSchedulingInfo.getUser(), this.rmContext,
|
appSchedulingInfo.getUser(), this.rmContext,
|
||||||
request.getNodeLabelExpression());
|
request.getNodeLabelExpression());
|
||||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||||
@ -76,7 +76,7 @@ public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
|||||||
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
||||||
|
|
||||||
// Add it to allContainers list.
|
// Add it to allContainers list.
|
||||||
newlyAllocatedContainers.add(rmContainer);
|
addToNewlyAllocatedContainers(node, rmContainer);
|
||||||
|
|
||||||
ContainerId containerId = container.getId();
|
ContainerId containerId = container.getId();
|
||||||
liveContainers.put(containerId, rmContainer);
|
liveContainers.put(containerId, rmContainer);
|
||||||
|
@ -49,7 +49,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
@ -71,6 +70,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
@ -326,8 +326,7 @@ public synchronized void setRMContext(RMContext rmContext) {
|
|||||||
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
|
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
|
||||||
List<ResourceRequest> ask, List<ContainerId> release,
|
List<ResourceRequest> ask, List<ContainerId> release,
|
||||||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||||
List<UpdateContainerRequest> increaseRequests,
|
ContainerUpdates updateRequests) {
|
||||||
List<UpdateContainerRequest> decreaseRequests) {
|
|
||||||
FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
|
FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.error("Calling allocate on removed " +
|
LOG.error("Calling allocate on removed " +
|
||||||
|
@ -157,7 +157,8 @@ public ResourceRequest getResourceRequest(String resourceName) {
|
|||||||
return resourceRequestMap.get(resourceName);
|
return resourceRequestMap.get(resourceName);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decrementOutstanding(ResourceRequest offSwitchRequest) {
|
private void decrementOutstanding(SchedulerRequestKey schedulerRequestKey,
|
||||||
|
ResourceRequest offSwitchRequest) {
|
||||||
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
|
||||||
|
|
||||||
// Do not remove ANY
|
// Do not remove ANY
|
||||||
@ -166,8 +167,6 @@ private void decrementOutstanding(ResourceRequest offSwitchRequest) {
|
|||||||
// Do we have any outstanding requests?
|
// Do we have any outstanding requests?
|
||||||
// If there is nothing, we need to deactivate this application
|
// If there is nothing, we need to deactivate this application
|
||||||
if (numOffSwitchContainers == 0) {
|
if (numOffSwitchContainers == 0) {
|
||||||
SchedulerRequestKey schedulerRequestKey = SchedulerRequestKey.create(
|
|
||||||
offSwitchRequest);
|
|
||||||
appSchedulingInfo.decrementSchedulerKeyReference(schedulerRequestKey);
|
appSchedulingInfo.decrementSchedulerKeyReference(schedulerRequestKey);
|
||||||
appSchedulingInfo.checkForDeactivation();
|
appSchedulingInfo.checkForDeactivation();
|
||||||
}
|
}
|
||||||
@ -177,11 +176,15 @@ private void decrementOutstanding(ResourceRequest offSwitchRequest) {
|
|||||||
offSwitchRequest.getCapability());
|
offSwitchRequest.getCapability());
|
||||||
}
|
}
|
||||||
|
|
||||||
private ResourceRequest cloneResourceRequest(ResourceRequest request) {
|
public ResourceRequest cloneResourceRequest(ResourceRequest request) {
|
||||||
ResourceRequest newRequest =
|
ResourceRequest newRequest = ResourceRequest.newBuilder()
|
||||||
ResourceRequest.newInstance(request.getPriority(),
|
.priority(request.getPriority())
|
||||||
request.getResourceName(), request.getCapability(), 1,
|
.allocationRequestId(request.getAllocationRequestId())
|
||||||
request.getRelaxLocality(), request.getNodeLabelExpression());
|
.resourceName(request.getResourceName())
|
||||||
|
.capability(request.getCapability())
|
||||||
|
.numContainers(1)
|
||||||
|
.relaxLocality(request.getRelaxLocality())
|
||||||
|
.nodeLabelExpression(request.getNodeLabelExpression()).build();
|
||||||
return newRequest;
|
return newRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -189,15 +192,15 @@ private ResourceRequest cloneResourceRequest(ResourceRequest request) {
|
|||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private void allocateRackLocal(SchedulerNode node,
|
private void allocateRackLocal(SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest rackLocalRequest,
|
SchedulerNode node, ResourceRequest rackLocalRequest,
|
||||||
List<ResourceRequest> resourceRequests) {
|
List<ResourceRequest> resourceRequests) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
decResourceRequest(node.getRackName(), rackLocalRequest);
|
decResourceRequest(node.getRackName(), rackLocalRequest);
|
||||||
|
|
||||||
ResourceRequest offRackRequest = resourceRequestMap.get(
|
ResourceRequest offRackRequest = resourceRequestMap.get(
|
||||||
ResourceRequest.ANY);
|
ResourceRequest.ANY);
|
||||||
decrementOutstanding(offRackRequest);
|
decrementOutstanding(schedulerKey, offRackRequest);
|
||||||
|
|
||||||
// Update cloned RackLocal and OffRack requests for recovery
|
// Update cloned RackLocal and OffRack requests for recovery
|
||||||
resourceRequests.add(cloneResourceRequest(rackLocalRequest));
|
resourceRequests.add(cloneResourceRequest(rackLocalRequest));
|
||||||
@ -208,10 +211,11 @@ private void allocateRackLocal(SchedulerNode node,
|
|||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private void allocateOffSwitch(ResourceRequest offSwitchRequest,
|
private void allocateOffSwitch(SchedulerRequestKey schedulerKey,
|
||||||
|
ResourceRequest offSwitchRequest,
|
||||||
List<ResourceRequest> resourceRequests) {
|
List<ResourceRequest> resourceRequests) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
decrementOutstanding(offSwitchRequest);
|
decrementOutstanding(schedulerKey, offSwitchRequest);
|
||||||
// Update cloned OffRack requests for recovery
|
// Update cloned OffRack requests for recovery
|
||||||
resourceRequests.add(cloneResourceRequest(offSwitchRequest));
|
resourceRequests.add(cloneResourceRequest(offSwitchRequest));
|
||||||
}
|
}
|
||||||
@ -221,8 +225,8 @@ private void allocateOffSwitch(ResourceRequest offSwitchRequest,
|
|||||||
* The {@link ResourceScheduler} is allocating data-local resources to the
|
* The {@link ResourceScheduler} is allocating data-local resources to the
|
||||||
* application.
|
* application.
|
||||||
*/
|
*/
|
||||||
private void allocateNodeLocal(SchedulerNode node,
|
private void allocateNodeLocal(SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest nodeLocalRequest,
|
SchedulerNode node, ResourceRequest nodeLocalRequest,
|
||||||
List<ResourceRequest> resourceRequests) {
|
List<ResourceRequest> resourceRequests) {
|
||||||
// Update future requirements
|
// Update future requirements
|
||||||
decResourceRequest(node.getNodeName(), nodeLocalRequest);
|
decResourceRequest(node.getNodeName(), nodeLocalRequest);
|
||||||
@ -233,7 +237,7 @@ private void allocateNodeLocal(SchedulerNode node,
|
|||||||
|
|
||||||
ResourceRequest offRackRequest = resourceRequestMap.get(
|
ResourceRequest offRackRequest = resourceRequestMap.get(
|
||||||
ResourceRequest.ANY);
|
ResourceRequest.ANY);
|
||||||
decrementOutstanding(offRackRequest);
|
decrementOutstanding(schedulerKey, offRackRequest);
|
||||||
|
|
||||||
// Update cloned NodeLocal, RackLocal and OffRack requests for recovery
|
// Update cloned NodeLocal, RackLocal and OffRack requests for recovery
|
||||||
resourceRequests.add(cloneResourceRequest(nodeLocalRequest));
|
resourceRequests.add(cloneResourceRequest(nodeLocalRequest));
|
||||||
@ -278,8 +282,8 @@ public boolean canAllocate(NodeType type, SchedulerNode node) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest request) {
|
NodeType type, SchedulerNode node, ResourceRequest request) {
|
||||||
try {
|
try {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
|
|
||||||
@ -296,11 +300,11 @@ public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (type == NodeType.NODE_LOCAL) {
|
if (type == NodeType.NODE_LOCAL) {
|
||||||
allocateNodeLocal(node, request, resourceRequests);
|
allocateNodeLocal(schedulerKey, node, request, resourceRequests);
|
||||||
} else if (type == NodeType.RACK_LOCAL) {
|
} else if (type == NodeType.RACK_LOCAL) {
|
||||||
allocateRackLocal(node, request, resourceRequests);
|
allocateRackLocal(schedulerKey, node, request, resourceRequests);
|
||||||
} else{
|
} else{
|
||||||
allocateOffSwitch(request, resourceRequests);
|
allocateOffSwitch(schedulerKey, request, resourceRequests);
|
||||||
}
|
}
|
||||||
|
|
||||||
return resourceRequests;
|
return resourceRequests;
|
||||||
|
@ -78,13 +78,14 @@ ResourceRequestUpdateResult updateResourceRequests(
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Notify container allocated.
|
* Notify container allocated.
|
||||||
|
* @param schedulerKey SchedulerRequestKey for this ResourceRequest
|
||||||
* @param type Type of the allocation
|
* @param type Type of the allocation
|
||||||
* @param node Which node this container allocated on
|
* @param node Which node this container allocated on
|
||||||
* @param request Which resource request to allocate
|
* @param request Which resource request to allocate
|
||||||
* @return list of ResourceRequests deducted
|
* @return list of ResourceRequests deducted
|
||||||
*/
|
*/
|
||||||
List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
|
||||||
ResourceRequest request);
|
NodeType type, SchedulerNode node, ResourceRequest request);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* We can still have pending requirement for a given NodeType and node
|
* We can still have pending requirement for a given NodeType and node
|
||||||
|
@ -58,6 +58,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
|
|
||||||
@ -331,7 +332,8 @@ public synchronized List<Container> getResources() throws IOException {
|
|||||||
// Get resources from the ResourceManager
|
// Get resources from the ResourceManager
|
||||||
Allocation allocation = resourceManager.getResourceScheduler().allocate(
|
Allocation allocation = resourceManager.getResourceScheduler().allocate(
|
||||||
applicationAttemptId, new ArrayList<ResourceRequest>(ask),
|
applicationAttemptId, new ArrayList<ResourceRequest>(ask),
|
||||||
new ArrayList<ContainerId>(), null, null, null, null);
|
new ArrayList<ContainerId>(), null, null,
|
||||||
|
new ContainerUpdates());
|
||||||
|
|
||||||
if (LOG.isInfoEnabled()) {
|
if (LOG.isInfoEnabled()) {
|
||||||
LOG.info("-=======" + applicationAttemptId + System.lineSeparator() +
|
LOG.info("-=======" + applicationAttemptId + System.lineSeparator() +
|
||||||
|
@ -251,6 +251,13 @@ public AllocateResponse sendContainerResizingRequest(
|
|||||||
return allocate(req);
|
return allocate(req);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public AllocateResponse sendContainerUpdateRequest(
|
||||||
|
List<UpdateContainerRequest> updateRequests) throws Exception {
|
||||||
|
final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
|
||||||
|
null, updateRequests);
|
||||||
|
return allocate(req);
|
||||||
|
}
|
||||||
|
|
||||||
public AllocateResponse allocate(AllocateRequest allocateRequest)
|
public AllocateResponse allocate(AllocateRequest allocateRequest)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
UserGroupInformation ugi =
|
UserGroupInformation ugi =
|
||||||
|
@ -195,6 +195,12 @@ public NodeHeartbeatResponse nodeHeartbeat(Map<ApplicationId,
|
|||||||
isHealthy, resId);
|
isHealthy, resId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public NodeHeartbeatResponse nodeHeartbeat(
|
||||||
|
List<ContainerStatus> updatedStats, boolean isHealthy) throws Exception {
|
||||||
|
return nodeHeartbeat(updatedStats, Collections.<Container>emptyList(),
|
||||||
|
isHealthy, ++responseId);
|
||||||
|
}
|
||||||
|
|
||||||
public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
|
public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
|
||||||
List<Container> increasedConts, boolean isHealthy, int resId)
|
List<Container> increasedConts, boolean isHealthy, int resId)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
|
@ -142,6 +142,8 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.Clock;
|
import org.apache.hadoop.yarn.util.Clock;
|
||||||
@ -1072,7 +1074,8 @@ public ApplicationReport createAndGetApplicationReport(
|
|||||||
Container container = Container.newInstance(
|
Container container = Container.newInstance(
|
||||||
ContainerId.newContainerId(attemptId, 1), null, "", null, null, null);
|
ContainerId.newContainerId(attemptId, 1), null, "", null, null, null);
|
||||||
RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
|
RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
|
||||||
attemptId, null, "", rmContext));
|
SchedulerRequestKey.extractFrom(container), attemptId, null, "",
|
||||||
|
rmContext));
|
||||||
Map<ApplicationAttemptId, RMAppAttempt> attempts =
|
Map<ApplicationAttemptId, RMAppAttempt> attempts =
|
||||||
new HashMap<ApplicationAttemptId, RMAppAttempt>();
|
new HashMap<ApplicationAttemptId, RMAppAttempt>();
|
||||||
attempts.put(attemptId, rmAppAttemptImpl);
|
attempts.put(attemptId, rmAppAttemptImpl);
|
||||||
|
@ -34,11 +34,15 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
|
||||||
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
||||||
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
|
import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
@ -64,8 +68,11 @@
|
|||||||
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
|
||||||
|
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.rmnode.RMNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
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.CapacityScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
||||||
@ -75,13 +82,17 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
||||||
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -91,8 +102,10 @@ public class TestOpportunisticContainerAllocatorAMService {
|
|||||||
|
|
||||||
private static final int GB = 1024;
|
private static final int GB = 1024;
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
private MockRM rm;
|
||||||
public void testNodeRemovalDuringAllocate() throws Exception {
|
|
||||||
|
@Before
|
||||||
|
public void createAndStartRM() {
|
||||||
CapacitySchedulerConfiguration csConf =
|
CapacitySchedulerConfiguration csConf =
|
||||||
new CapacitySchedulerConfiguration();
|
new CapacitySchedulerConfiguration();
|
||||||
YarnConfiguration conf = new YarnConfiguration(csConf);
|
YarnConfiguration conf = new YarnConfiguration(csConf);
|
||||||
@ -102,8 +115,445 @@ public void testNodeRemovalDuringAllocate() throws Exception {
|
|||||||
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
|
||||||
conf.setInt(
|
conf.setInt(
|
||||||
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
|
||||||
MockRM rm = new MockRM(conf);
|
rm = new MockRM(conf);
|
||||||
rm.start();
|
rm.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void stopRM() {
|
||||||
|
if (rm != null) {
|
||||||
|
rm.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 600000)
|
||||||
|
public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception {
|
||||||
|
HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
||||||
|
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm1.getNodeId(), nm1);
|
||||||
|
MockNM nm2 = new MockNM("h1:4321", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm2.getNodeId(), nm2);
|
||||||
|
MockNM nm3 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm3.getNodeId(), nm3);
|
||||||
|
MockNM nm4 = new MockNM("h2:4321", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm4.getNodeId(), nm4);
|
||||||
|
nm1.registerNode();
|
||||||
|
nm2.registerNode();
|
||||||
|
nm3.registerNode();
|
||||||
|
nm4.registerNode();
|
||||||
|
|
||||||
|
OpportunisticContainerAllocatorAMService amservice =
|
||||||
|
(OpportunisticContainerAllocatorAMService) rm
|
||||||
|
.getApplicationMasterService();
|
||||||
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
||||||
|
ApplicationAttemptId attemptId =
|
||||||
|
app1.getCurrentAppAttempt().getAppAttemptId();
|
||||||
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
|
||||||
|
ResourceScheduler scheduler = rm.getResourceScheduler();
|
||||||
|
RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
||||||
|
RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
||||||
|
RMNode rmNode3 = rm.getRMContext().getRMNodes().get(nm3.getNodeId());
|
||||||
|
RMNode rmNode4 = rm.getRMContext().getRMNodes().get(nm4.getNodeId());
|
||||||
|
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
nm3.nodeHeartbeat(true);
|
||||||
|
nm4.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
((RMNodeImpl) rmNode1)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
((RMNodeImpl) rmNode2)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
((RMNodeImpl) rmNode3)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
((RMNodeImpl) rmNode4)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
|
||||||
|
OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(attemptId).getOpportunisticContainerContext();
|
||||||
|
// Send add and update node events to AM Service.
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode3));
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode4));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode3));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode4));
|
||||||
|
// All nodes 1 - 4 will be applicable for scheduling.
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
nm3.nodeHeartbeat(true);
|
||||||
|
nm4.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
Thread.sleep(1000);
|
||||||
|
|
||||||
|
QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
|
||||||
|
.getMetrics();
|
||||||
|
|
||||||
|
// Verify Metrics
|
||||||
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
||||||
|
|
||||||
|
AllocateResponse allocateResponse = am1.allocate(
|
||||||
|
Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
|
||||||
|
"*", Resources.createResource(1 * GB), 2, true, null,
|
||||||
|
ExecutionTypeRequest.newInstance(
|
||||||
|
ExecutionType.OPPORTUNISTIC, true))),
|
||||||
|
null);
|
||||||
|
List<Container> allocatedContainers = allocateResponse
|
||||||
|
.getAllocatedContainers();
|
||||||
|
Assert.assertEquals(2, allocatedContainers.size());
|
||||||
|
Container container = allocatedContainers.get(0);
|
||||||
|
MockNM allocNode = nodes.get(container.getNodeId());
|
||||||
|
MockNM sameHostDiffNode = null;
|
||||||
|
for (NodeId n : nodes.keySet()) {
|
||||||
|
if (n.getHost().equals(allocNode.getNodeId().getHost()) &&
|
||||||
|
n.getPort() != allocNode.getNodeId().getPort()) {
|
||||||
|
sameHostDiffNode = nodes.get(n);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change)
|
||||||
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
||||||
|
|
||||||
|
am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(0,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
// Node on same host should not result in allocation
|
||||||
|
sameHostDiffNode.nodeHeartbeat(true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
||||||
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change again)
|
||||||
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
||||||
|
|
||||||
|
// Send Promotion req again... this should result in update error
|
||||||
|
allocateResponse = am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(0,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
|
||||||
|
Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
|
||||||
|
allocateResponse.getUpdateErrors().get(0).getReason());
|
||||||
|
Assert.assertEquals(container.getId(),
|
||||||
|
allocateResponse.getUpdateErrors().get(0)
|
||||||
|
.getUpdateContainerRequest().getContainerId());
|
||||||
|
|
||||||
|
// Send Promotion req again with incorrect version...
|
||||||
|
// this should also result in update error
|
||||||
|
allocateResponse = am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(1,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
|
||||||
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
|
||||||
|
Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|1|0",
|
||||||
|
allocateResponse.getUpdateErrors().get(0).getReason());
|
||||||
|
Assert.assertEquals(container.getId(),
|
||||||
|
allocateResponse.getUpdateErrors().get(0)
|
||||||
|
.getUpdateContainerRequest().getContainerId());
|
||||||
|
|
||||||
|
// Ensure after correct node heartbeats, we should get the allocation
|
||||||
|
allocNode.nodeHeartbeat(true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Container uc =
|
||||||
|
allocateResponse.getUpdatedContainers().get(0).getContainer();
|
||||||
|
Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
|
||||||
|
Assert.assertEquals(uc.getId(), container.getId());
|
||||||
|
Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation :
|
||||||
|
// Allocated cores+mem should have increased, available should decrease
|
||||||
|
verifyMetrics(metrics, 14336, 14, 2048, 2, 2);
|
||||||
|
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
nm3.nodeHeartbeat(true);
|
||||||
|
nm4.nodeHeartbeat(true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
|
||||||
|
// Verify that the container is still in ACQUIRED state wrt the RM.
|
||||||
|
RMContainer rmContainer = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(
|
||||||
|
uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
|
||||||
|
Assert.assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
|
||||||
|
|
||||||
|
// Now demote the container back..
|
||||||
|
allocateResponse = am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(uc.getVersion(),
|
||||||
|
uc.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.OPPORTUNISTIC)));
|
||||||
|
// This should happen in the same heartbeat..
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
||||||
|
uc = allocateResponse.getUpdatedContainers().get(0).getContainer();
|
||||||
|
Assert.assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType());
|
||||||
|
Assert.assertEquals(uc.getId(), container.getId());
|
||||||
|
Assert.assertEquals(uc.getVersion(), container.getVersion() + 2);
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation :
|
||||||
|
// Everything should have reverted to what it was
|
||||||
|
verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testContainerPromoteAfterContainerStart() throws Exception {
|
||||||
|
HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
||||||
|
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm1.getNodeId(), nm1);
|
||||||
|
MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm2.getNodeId(), nm2);
|
||||||
|
nm1.registerNode();
|
||||||
|
nm2.registerNode();
|
||||||
|
|
||||||
|
OpportunisticContainerAllocatorAMService amservice =
|
||||||
|
(OpportunisticContainerAllocatorAMService) rm
|
||||||
|
.getApplicationMasterService();
|
||||||
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
||||||
|
ApplicationAttemptId attemptId =
|
||||||
|
app1.getCurrentAppAttempt().getAppAttemptId();
|
||||||
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
|
||||||
|
ResourceScheduler scheduler = rm.getResourceScheduler();
|
||||||
|
RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
||||||
|
RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
||||||
|
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
((RMNodeImpl) rmNode1)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
((RMNodeImpl) rmNode2)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
|
||||||
|
OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(attemptId).getOpportunisticContainerContext();
|
||||||
|
// Send add and update node events to AM Service.
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
|
||||||
|
|
||||||
|
// All nodes 1 to 2 will be applicable for scheduling.
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
Thread.sleep(1000);
|
||||||
|
|
||||||
|
QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
|
||||||
|
.getMetrics();
|
||||||
|
|
||||||
|
// Verify Metrics
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
|
||||||
|
AllocateResponse allocateResponse = am1.allocate(
|
||||||
|
Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
|
||||||
|
"*", Resources.createResource(1 * GB), 2, true, null,
|
||||||
|
ExecutionTypeRequest.newInstance(
|
||||||
|
ExecutionType.OPPORTUNISTIC, true))),
|
||||||
|
null);
|
||||||
|
List<Container> allocatedContainers = allocateResponse
|
||||||
|
.getAllocatedContainers();
|
||||||
|
Assert.assertEquals(2, allocatedContainers.size());
|
||||||
|
Container container = allocatedContainers.get(0);
|
||||||
|
MockNM allocNode = nodes.get(container.getNodeId());
|
||||||
|
|
||||||
|
// Start Container in NM
|
||||||
|
allocNode.nodeHeartbeat(Arrays.asList(
|
||||||
|
ContainerStatus.newInstance(container.getId(),
|
||||||
|
ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
|
||||||
|
true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
|
||||||
|
// Verify that container is actually running wrt the RM..
|
||||||
|
RMContainer rmContainer = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(
|
||||||
|
container.getId().getApplicationAttemptId()).getRMContainer(
|
||||||
|
container.getId());
|
||||||
|
Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change)
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
|
||||||
|
am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(0,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change again)
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
|
||||||
|
// Send Promotion req again... this should result in update error
|
||||||
|
allocateResponse = am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(0,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
|
||||||
|
Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
|
||||||
|
allocateResponse.getUpdateErrors().get(0).getReason());
|
||||||
|
Assert.assertEquals(container.getId(),
|
||||||
|
allocateResponse.getUpdateErrors().get(0)
|
||||||
|
.getUpdateContainerRequest().getContainerId());
|
||||||
|
|
||||||
|
// Start Container in NM
|
||||||
|
allocNode.nodeHeartbeat(Arrays.asList(
|
||||||
|
ContainerStatus.newInstance(container.getId(),
|
||||||
|
ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
|
||||||
|
true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
|
||||||
|
allocateResponse = am1.allocate(new ArrayList<>(), new ArrayList<>());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Container uc =
|
||||||
|
allocateResponse.getUpdatedContainers().get(0).getContainer();
|
||||||
|
Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
|
||||||
|
Assert.assertEquals(uc.getId(), container.getId());
|
||||||
|
Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
|
||||||
|
|
||||||
|
// Verify that the Container is still in RUNNING state wrt RM..
|
||||||
|
rmContainer = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(
|
||||||
|
uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
|
||||||
|
Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation :
|
||||||
|
// Allocated cores+mem should have increased, available should decrease
|
||||||
|
verifyMetrics(metrics, 6144, 6, 2048, 2, 2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 600000)
|
||||||
|
public void testContainerPromoteAfterContainerComplete() throws Exception {
|
||||||
|
HashMap<NodeId, MockNM> nodes = new HashMap<>();
|
||||||
|
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm1.getNodeId(), nm1);
|
||||||
|
MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
||||||
|
nodes.put(nm2.getNodeId(), nm2);
|
||||||
|
nm1.registerNode();
|
||||||
|
nm2.registerNode();
|
||||||
|
|
||||||
|
OpportunisticContainerAllocatorAMService amservice =
|
||||||
|
(OpportunisticContainerAllocatorAMService) rm
|
||||||
|
.getApplicationMasterService();
|
||||||
|
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
|
||||||
|
ApplicationAttemptId attemptId =
|
||||||
|
app1.getCurrentAppAttempt().getAppAttemptId();
|
||||||
|
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
|
||||||
|
ResourceScheduler scheduler = rm.getResourceScheduler();
|
||||||
|
RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
|
||||||
|
RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
|
||||||
|
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
((RMNodeImpl) rmNode1)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
((RMNodeImpl) rmNode2)
|
||||||
|
.setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
|
||||||
|
|
||||||
|
OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(attemptId).getOpportunisticContainerContext();
|
||||||
|
// Send add and update node events to AM Service.
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
||||||
|
amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
|
||||||
|
|
||||||
|
// All nodes 1 to 2 will be applicable for scheduling.
|
||||||
|
nm1.nodeHeartbeat(true);
|
||||||
|
nm2.nodeHeartbeat(true);
|
||||||
|
|
||||||
|
Thread.sleep(1000);
|
||||||
|
|
||||||
|
QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
|
||||||
|
.getMetrics();
|
||||||
|
|
||||||
|
// Verify Metrics
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
|
||||||
|
AllocateResponse allocateResponse = am1.allocate(
|
||||||
|
Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
|
||||||
|
"*", Resources.createResource(1 * GB), 2, true, null,
|
||||||
|
ExecutionTypeRequest.newInstance(
|
||||||
|
ExecutionType.OPPORTUNISTIC, true))),
|
||||||
|
null);
|
||||||
|
List<Container> allocatedContainers = allocateResponse
|
||||||
|
.getAllocatedContainers();
|
||||||
|
Assert.assertEquals(2, allocatedContainers.size());
|
||||||
|
Container container = allocatedContainers.get(0);
|
||||||
|
MockNM allocNode = nodes.get(container.getNodeId());
|
||||||
|
|
||||||
|
// Start Container in NM
|
||||||
|
allocNode.nodeHeartbeat(Arrays.asList(
|
||||||
|
ContainerStatus.newInstance(container.getId(),
|
||||||
|
ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
|
||||||
|
true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
|
||||||
|
// Verify that container is actually running wrt the RM..
|
||||||
|
RMContainer rmContainer = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(
|
||||||
|
container.getId().getApplicationAttemptId()).getRMContainer(
|
||||||
|
container.getId());
|
||||||
|
Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
|
||||||
|
|
||||||
|
// Container Completed in the NM
|
||||||
|
allocNode.nodeHeartbeat(Arrays.asList(
|
||||||
|
ContainerStatus.newInstance(container.getId(),
|
||||||
|
ExecutionType.OPPORTUNISTIC, ContainerState.COMPLETE, "", 0)),
|
||||||
|
true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
|
||||||
|
// Verify that container has been removed..
|
||||||
|
rmContainer = ((CapacityScheduler) scheduler)
|
||||||
|
.getApplicationAttempt(
|
||||||
|
container.getId().getApplicationAttemptId()).getRMContainer(
|
||||||
|
container.getId());
|
||||||
|
Assert.assertNull(rmContainer);
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change)
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
|
||||||
|
// Send Promotion req... this should result in update error
|
||||||
|
// Since the container doesn't exist anymore..
|
||||||
|
allocateResponse = am1.sendContainerUpdateRequest(
|
||||||
|
Arrays.asList(UpdateContainerRequest.newInstance(0,
|
||||||
|
container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
|
||||||
|
null, ExecutionType.GUARANTEED)));
|
||||||
|
|
||||||
|
Assert.assertEquals(1,
|
||||||
|
allocateResponse.getCompletedContainersStatuses().size());
|
||||||
|
Assert.assertEquals(container.getId(),
|
||||||
|
allocateResponse.getCompletedContainersStatuses().get(0)
|
||||||
|
.getContainerId());
|
||||||
|
Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
|
||||||
|
Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
|
||||||
|
Assert.assertEquals("INVALID_CONTAINER_ID",
|
||||||
|
allocateResponse.getUpdateErrors().get(0).getReason());
|
||||||
|
Assert.assertEquals(container.getId(),
|
||||||
|
allocateResponse.getUpdateErrors().get(0)
|
||||||
|
.getUpdateContainerRequest().getContainerId());
|
||||||
|
|
||||||
|
// Verify Metrics After OPP allocation (Nothing should change again)
|
||||||
|
verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void verifyMetrics(QueueMetrics metrics, long availableMB,
|
||||||
|
int availableVirtualCores, long allocatedMB,
|
||||||
|
int allocatedVirtualCores, int allocatedContainers) {
|
||||||
|
Assert.assertEquals(availableMB, metrics.getAvailableMB());
|
||||||
|
Assert.assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores());
|
||||||
|
Assert.assertEquals(allocatedMB, metrics.getAllocatedMB());
|
||||||
|
Assert.assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores());
|
||||||
|
Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testNodeRemovalDuringAllocate() throws Exception {
|
||||||
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
|
||||||
MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
|
||||||
nm1.registerNode();
|
nm1.registerNode();
|
||||||
|
@ -100,6 +100,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
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.SchedulerUtils;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||||
@ -478,7 +479,7 @@ private void testAppAttemptScheduledState() {
|
|||||||
assertEquals(expectedState, applicationAttempt.getAppAttemptState());
|
assertEquals(expectedState, applicationAttempt.getAppAttemptState());
|
||||||
verify(scheduler, times(expectedAllocateCount)).allocate(
|
verify(scheduler, times(expectedAllocateCount)).allocate(
|
||||||
any(ApplicationAttemptId.class), any(List.class), any(List.class),
|
any(ApplicationAttemptId.class), any(List.class), any(List.class),
|
||||||
any(List.class), any(List.class), any(List.class), any(List.class));
|
any(List.class), any(List.class), any(ContainerUpdates.class));
|
||||||
|
|
||||||
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
|
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
|
||||||
assertNull(applicationAttempt.getMasterContainer());
|
assertNull(applicationAttempt.getMasterContainer());
|
||||||
@ -499,7 +500,7 @@ private void testAppAttemptAllocatedState(Container amContainer) {
|
|||||||
verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
|
verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
|
||||||
verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.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), any(List.class), any(List.class),
|
||||||
any(List.class), any(List.class));
|
any(ContainerUpdates.class));
|
||||||
verify(nmTokenManager).clearNodeSetForAttempt(
|
verify(nmTokenManager).clearNodeSetForAttempt(
|
||||||
applicationAttempt.getAppAttemptId());
|
applicationAttempt.getAppAttemptId());
|
||||||
}
|
}
|
||||||
@ -526,7 +527,7 @@ private void testAppAttemptFailedState(Container container,
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link RMAppAttemptState#LAUNCH}
|
* {@link RMAppAttemptState#LAUNCHED}
|
||||||
*/
|
*/
|
||||||
private void testAppAttemptLaunchedState(Container container) {
|
private void testAppAttemptLaunchedState(Container container) {
|
||||||
assertEquals(RMAppAttemptState.LAUNCHED,
|
assertEquals(RMAppAttemptState.LAUNCHED,
|
||||||
@ -649,8 +650,8 @@ private Container allocateApplicationAttempt() {
|
|||||||
when(allocation.getContainers()).
|
when(allocation.getContainers()).
|
||||||
thenReturn(Collections.singletonList(container));
|
thenReturn(Collections.singletonList(container));
|
||||||
when(scheduler.allocate(any(ApplicationAttemptId.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), any(List.class), any(List.class),
|
||||||
any(List.class))).
|
any(ContainerUpdates.class))).
|
||||||
thenReturn(allocation);
|
thenReturn(allocation);
|
||||||
RMContainer rmContainer = mock(RMContainerImpl.class);
|
RMContainer rmContainer = mock(RMContainerImpl.class);
|
||||||
when(scheduler.getRMContainer(container.getId())).
|
when(scheduler.getRMContainer(container.getId())).
|
||||||
@ -1129,8 +1130,9 @@ public void testLaunchedFailWhileAHSEnabled() {
|
|||||||
when(allocation.getContainers()).
|
when(allocation.getContainers()).
|
||||||
thenReturn(Collections.singletonList(amContainer));
|
thenReturn(Collections.singletonList(amContainer));
|
||||||
when(scheduler.allocate(any(ApplicationAttemptId.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), any(List.class), any(List.class),
|
||||||
any(List.class))).thenReturn(allocation);
|
any(ContainerUpdates.class)))
|
||||||
|
.thenReturn(allocation);
|
||||||
RMContainer rmContainer = mock(RMContainerImpl.class);
|
RMContainer rmContainer = mock(RMContainerImpl.class);
|
||||||
when(scheduler.getRMContainer(amContainer.getId())).thenReturn(rmContainer);
|
when(scheduler.getRMContainer(amContainer.getId())).thenReturn(rmContainer);
|
||||||
|
|
||||||
@ -1610,7 +1612,8 @@ public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() {
|
|||||||
YarnScheduler mockScheduler = mock(YarnScheduler.class);
|
YarnScheduler mockScheduler = mock(YarnScheduler.class);
|
||||||
when(mockScheduler.allocate(any(ApplicationAttemptId.class),
|
when(mockScheduler.allocate(any(ApplicationAttemptId.class),
|
||||||
any(List.class), any(List.class), any(List.class), any(List.class),
|
any(List.class), any(List.class), any(List.class), any(List.class),
|
||||||
any(List.class), any(List.class))).thenAnswer(new Answer<Allocation>() {
|
any(ContainerUpdates.class)))
|
||||||
|
.thenAnswer(new Answer<Allocation>() {
|
||||||
|
|
||||||
@SuppressWarnings("rawtypes")
|
@SuppressWarnings("rawtypes")
|
||||||
@Override
|
@Override
|
||||||
|
@ -62,6 +62,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
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.SchedulerUtils;
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
@ -114,7 +115,8 @@ public void testReleaseWhileRunning() {
|
|||||||
YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
|
YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
|
||||||
true);
|
true);
|
||||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container), appAttemptId,
|
||||||
nodeId, "user", rmContext);
|
nodeId, "user", rmContext);
|
||||||
|
|
||||||
assertEquals(RMContainerState.NEW, rmContainer.getState());
|
assertEquals(RMContainerState.NEW, rmContainer.getState());
|
||||||
@ -216,7 +218,8 @@ public void testExpireWhileRunning() {
|
|||||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||||
when(rmContext.getRMApps()).thenReturn(appMap);
|
when(rmContext.getRMApps()).thenReturn(appMap);
|
||||||
|
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container), appAttemptId,
|
||||||
nodeId, "user", rmContext);
|
nodeId, "user", rmContext);
|
||||||
|
|
||||||
assertEquals(RMContainerState.NEW, rmContainer.getState());
|
assertEquals(RMContainerState.NEW, rmContainer.getState());
|
||||||
|
@ -114,6 +114,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.
|
||||||
ContainerExpiredSchedulerEvent;
|
ContainerExpiredSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
@ -160,7 +161,9 @@
|
|||||||
public class TestCapacityScheduler {
|
public class TestCapacityScheduler {
|
||||||
private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
|
private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
|
||||||
private final int GB = 1024;
|
private final int GB = 1024;
|
||||||
|
private final static ContainerUpdates NULL_UPDATE_REQUESTS =
|
||||||
|
new ContainerUpdates();
|
||||||
|
|
||||||
private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
|
private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
|
||||||
private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
|
private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
|
||||||
private static final String A1 = A + ".a1";
|
private static final String A1 = A + ".a1";
|
||||||
@ -738,12 +741,12 @@ public void testBlackListNodes() throws Exception {
|
|||||||
// Verify the blacklist can be updated independent of requesting containers
|
// Verify the blacklist can be updated independent of requesting containers
|
||||||
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
Collections.singletonList(host), null, null, null);
|
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
|
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
|
||||||
.isPlaceBlacklisted(host));
|
.isPlaceBlacklisted(host));
|
||||||
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(), null,
|
Collections.<ContainerId>emptyList(), null,
|
||||||
Collections.singletonList(host), null, null);
|
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
|
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
|
||||||
.isPlaceBlacklisted(host));
|
.isPlaceBlacklisted(host));
|
||||||
rm.stop();
|
rm.stop();
|
||||||
@ -839,7 +842,7 @@ public void testAllocateReorder() throws Exception {
|
|||||||
cs.allocate(appAttemptId1,
|
cs.allocate(appAttemptId1,
|
||||||
Collections.<ResourceRequest>singletonList(r1),
|
Collections.<ResourceRequest>singletonList(r1),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
//And this will result in container assignment for app1
|
//And this will result in container assignment for app1
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
@ -856,7 +859,7 @@ public void testAllocateReorder() throws Exception {
|
|||||||
cs.allocate(appAttemptId2,
|
cs.allocate(appAttemptId2,
|
||||||
Collections.<ResourceRequest>singletonList(r2),
|
Collections.<ResourceRequest>singletonList(r2),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
//In this case we do not perform container assignment because we want to
|
//In this case we do not perform container assignment because we want to
|
||||||
//verify re-ordering based on the allocation alone
|
//verify re-ordering based on the allocation alone
|
||||||
@ -2981,7 +2984,8 @@ public void testApplicationHeadRoom() throws Exception {
|
|||||||
|
|
||||||
Allocation allocate =
|
Allocation allocate =
|
||||||
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
||||||
Collections.<ContainerId> emptyList(), null, null, null, null);
|
Collections.<ContainerId> emptyList(), null, null,
|
||||||
|
NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
Assert.assertNotNull(attempt);
|
Assert.assertNotNull(attempt);
|
||||||
|
|
||||||
@ -2997,7 +3001,8 @@ public void testApplicationHeadRoom() throws Exception {
|
|||||||
|
|
||||||
allocate =
|
allocate =
|
||||||
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
|
||||||
Collections.<ContainerId> emptyList(), null, null, null, null);
|
Collections.<ContainerId> emptyList(), null, null,
|
||||||
|
NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// All resources should be sent as headroom
|
// All resources should be sent as headroom
|
||||||
Assert.assertEquals(newResource, allocate.getResourceLimit());
|
Assert.assertEquals(newResource, allocate.getResourceLimit());
|
||||||
@ -3504,7 +3509,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId3,
|
cs.allocate(appAttemptId3,
|
||||||
Collections.<ResourceRequest>singletonList(y1Req),
|
Collections.<ResourceRequest>singletonList(y1Req),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
}
|
}
|
||||||
assertEquals("Y1 Used Resource should be 4 GB", 4 * GB,
|
assertEquals("Y1 Used Resource should be 4 GB", 4 * GB,
|
||||||
@ -3518,7 +3523,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId1,
|
cs.allocate(appAttemptId1,
|
||||||
Collections.<ResourceRequest>singletonList(x1Req),
|
Collections.<ResourceRequest>singletonList(x1Req),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
}
|
}
|
||||||
assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
|
assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
|
||||||
@ -3531,7 +3536,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId2,
|
cs.allocate(appAttemptId2,
|
||||||
Collections.<ResourceRequest>singletonList(x2Req),
|
Collections.<ResourceRequest>singletonList(x2Req),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
assertEquals("X2 Used Resource should be 0", 0,
|
assertEquals("X2 Used Resource should be 0", 0,
|
||||||
cs.getQueue("x2").getUsedResources().getMemorySize());
|
cs.getQueue("x2").getUsedResources().getMemorySize());
|
||||||
@ -3543,7 +3548,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId1,
|
cs.allocate(appAttemptId1,
|
||||||
Collections.<ResourceRequest>singletonList(x1Req),
|
Collections.<ResourceRequest>singletonList(x1Req),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
|
assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
|
||||||
cs.getQueue("x1").getUsedResources().getMemorySize());
|
cs.getQueue("x1").getUsedResources().getMemorySize());
|
||||||
@ -3557,7 +3562,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId3,
|
cs.allocate(appAttemptId3,
|
||||||
Collections.<ResourceRequest>singletonList(y1Req),
|
Collections.<ResourceRequest>singletonList(y1Req),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
}
|
}
|
||||||
assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
|
assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
|
||||||
@ -3616,7 +3621,7 @@ public void testCSQueueBlocked() throws Exception {
|
|||||||
//This will allocate for app1
|
//This will allocate for app1
|
||||||
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
|
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null).getContainers().size();
|
null, null, NULL_UPDATE_REQUESTS).getContainers().size();
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
ResourceRequest r2 = null;
|
ResourceRequest r2 = null;
|
||||||
for (int i =0; i < 13; i++) {
|
for (int i =0; i < 13; i++) {
|
||||||
@ -3625,7 +3630,7 @@ public void testCSQueueBlocked() throws Exception {
|
|||||||
cs.allocate(appAttemptId2,
|
cs.allocate(appAttemptId2,
|
||||||
Collections.<ResourceRequest>singletonList(r2),
|
Collections.<ResourceRequest>singletonList(r2),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null);
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
}
|
}
|
||||||
assertEquals("A Used Resource should be 2 GB", 2 * GB,
|
assertEquals("A Used Resource should be 2 GB", 2 * GB,
|
||||||
@ -3638,11 +3643,11 @@ public void testCSQueueBlocked() throws Exception {
|
|||||||
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
|
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
|
||||||
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
|
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
null, null, null, null).getContainers().size();
|
null, null, NULL_UPDATE_REQUESTS).getContainers().size();
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
|
|
||||||
cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
|
cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
|
||||||
Collections.<ContainerId>emptyList(), null, null, null, null);
|
Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
|
||||||
CapacityScheduler.schedule(cs);
|
CapacityScheduler.schedule(cs);
|
||||||
//Check blocked Resource
|
//Check blocked Resource
|
||||||
assertEquals("A Used Resource should be 2 GB", 2 * GB,
|
assertEquals("A Used Resource should be 2 GB", 2 * GB,
|
||||||
|
@ -55,6 +55,8 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
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.common.fica.FiCaSchedulerNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||||
@ -280,7 +282,8 @@ public void testSortedQueues() throws Exception {
|
|||||||
ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
|
ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
|
||||||
Container container=TestUtils.getMockContainer(containerId,
|
Container container=TestUtils.getMockContainer(containerId,
|
||||||
node_0.getNodeID(), Resources.createResource(1*GB), priority);
|
node_0.getNodeID(), Resources.createResource(1*GB), priority);
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container), appAttemptId,
|
||||||
node_0.getNodeID(), "user", rmContext);
|
node_0.getNodeID(), "user", rmContext);
|
||||||
|
|
||||||
// Assign {1,2,3,4} 1GB containers respectively to queues
|
// Assign {1,2,3,4} 1GB containers respectively to queues
|
||||||
|
@ -923,13 +923,15 @@ public void testGetAppToUnreserve() throws Exception {
|
|||||||
Container container = TestUtils.getMockContainer(containerId,
|
Container container = TestUtils.getMockContainer(containerId,
|
||||||
node_1.getNodeID(), Resources.createResource(2*GB),
|
node_1.getNodeID(), Resources.createResource(2*GB),
|
||||||
priorityMap.getPriority());
|
priorityMap.getPriority());
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container), appAttemptId,
|
||||||
node_1.getNodeID(), "user", rmContext);
|
node_1.getNodeID(), "user", rmContext);
|
||||||
|
|
||||||
Container container_1 = TestUtils.getMockContainer(containerId,
|
Container container_1 = TestUtils.getMockContainer(containerId,
|
||||||
node_0.getNodeID(), Resources.createResource(1*GB),
|
node_0.getNodeID(), Resources.createResource(1*GB),
|
||||||
priorityMap.getPriority());
|
priorityMap.getPriority());
|
||||||
RMContainer rmContainer_1 = new RMContainerImpl(container_1, appAttemptId,
|
RMContainer rmContainer_1 = new RMContainerImpl(container_1,
|
||||||
|
SchedulerRequestKey.extractFrom(container_1), appAttemptId,
|
||||||
node_0.getNodeID(), "user", rmContext);
|
node_0.getNodeID(), "user", rmContext);
|
||||||
|
|
||||||
// no reserved containers
|
// no reserved containers
|
||||||
@ -996,7 +998,8 @@ public void testFindNodeToUnreserve() throws Exception {
|
|||||||
Container container = TestUtils.getMockContainer(containerId,
|
Container container = TestUtils.getMockContainer(containerId,
|
||||||
node_1.getNodeID(), Resources.createResource(2*GB),
|
node_1.getNodeID(), Resources.createResource(2*GB),
|
||||||
priorityMap.getPriority());
|
priorityMap.getPriority());
|
||||||
RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
|
RMContainer rmContainer = new RMContainerImpl(container,
|
||||||
|
SchedulerRequestKey.extractFrom(container), appAttemptId,
|
||||||
node_1.getNodeID(), "user", rmContext);
|
node_1.getNodeID(), "user", rmContext);
|
||||||
|
|
||||||
// nothing reserved
|
// nothing reserved
|
||||||
|
@ -41,6 +41,9 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
||||||
@ -71,6 +74,8 @@ public class FairSchedulerTestBase {
|
|||||||
public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
|
public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
|
||||||
private static final int SLEEP_DURATION = 10;
|
private static final int SLEEP_DURATION = 10;
|
||||||
private static final int SLEEP_RETRIES = 1000;
|
private static final int SLEEP_RETRIES = 1000;
|
||||||
|
final static ContainerUpdates NULL_UPDATE_REQUESTS =
|
||||||
|
new ContainerUpdates();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The list of nodes added to the cluster using the {@link #addNode} method.
|
* The list of nodes added to the cluster using the {@link #addNode} method.
|
||||||
@ -181,7 +186,8 @@ protected ApplicationAttemptId createSchedulingRequest(
|
|||||||
resourceManager.getRMContext().getRMApps()
|
resourceManager.getRMContext().getRMApps()
|
||||||
.put(id.getApplicationId(), rmApp);
|
.put(id.getApplicationId(), rmApp);
|
||||||
|
|
||||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
return id;
|
return id;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -207,7 +213,8 @@ protected ApplicationAttemptId createSchedulingRequest(String queueId,
|
|||||||
resourceManager.getRMContext().getRMApps()
|
resourceManager.getRMContext().getRMApps()
|
||||||
.put(id.getApplicationId(), rmApp);
|
.put(id.getApplicationId(), rmApp);
|
||||||
|
|
||||||
scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
return id;
|
return id;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -229,7 +236,8 @@ protected void createSchedulingRequestExistingApplication(
|
|||||||
ResourceRequest request, ApplicationAttemptId attId) {
|
ResourceRequest request, ApplicationAttemptId attId) {
|
||||||
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
|
||||||
ask.add(request);
|
ask.add(request);
|
||||||
scheduler.allocate(attId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(attId, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void createApplicationWithAMResource(ApplicationAttemptId attId,
|
protected void createApplicationWithAMResource(ApplicationAttemptId attId,
|
||||||
|
@ -119,7 +119,8 @@ public void testBasic() throws InterruptedException {
|
|||||||
List<ResourceRequest> ask = new ArrayList<>();
|
List<ResourceRequest> ask = new ArrayList<>();
|
||||||
ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
|
ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
|
||||||
scheduler.allocate(
|
scheduler.allocate(
|
||||||
appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
appAttemptId, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
||||||
|
|
||||||
triggerSchedulingAttempt();
|
triggerSchedulingAttempt();
|
||||||
@ -157,7 +158,7 @@ public void testSortedNodes() throws Exception {
|
|||||||
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
|
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
|
||||||
ask.add(request);
|
ask.add(request);
|
||||||
scheduler.allocate(appAttemptId, ask,
|
scheduler.allocate(appAttemptId, ask,
|
||||||
new ArrayList<ContainerId>(), null, null, null, null);
|
new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
|
||||||
triggerSchedulingAttempt();
|
triggerSchedulingAttempt();
|
||||||
|
|
||||||
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
|
||||||
@ -169,7 +170,7 @@ public void testSortedNodes() throws Exception {
|
|||||||
ask.clear();
|
ask.clear();
|
||||||
ask.add(request);
|
ask.add(request);
|
||||||
scheduler.allocate(appAttemptId, ask,
|
scheduler.allocate(appAttemptId, ask,
|
||||||
new ArrayList<ContainerId>(), null, null, null, null);
|
new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
|
||||||
triggerSchedulingAttempt();
|
triggerSchedulingAttempt();
|
||||||
|
|
||||||
checkAppConsumption(app, Resources.createResource(2048,2));
|
checkAppConsumption(app, Resources.createResource(2048,2));
|
||||||
@ -335,7 +336,7 @@ public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
|
|||||||
ask1.add(request1);
|
ask1.add(request1);
|
||||||
ask1.add(request2);
|
ask1.add(request2);
|
||||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
|
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
|
||||||
null, null);
|
NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
|
||||||
scheduler.handle(nodeEvent1);
|
scheduler.handle(nodeEvent1);
|
||||||
|
@ -93,6 +93,9 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||||
@ -124,6 +127,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||||||
private final int GB = 1024;
|
private final int GB = 1024;
|
||||||
private final static String ALLOC_FILE =
|
private final static String ALLOC_FILE =
|
||||||
new File(TEST_DIR, "test-queues").getAbsolutePath();
|
new File(TEST_DIR, "test-queues").getAbsolutePath();
|
||||||
|
private final static ContainerUpdates NULL_UPDATE_REQUESTS =
|
||||||
|
new ContainerUpdates();
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws IOException {
|
public void setUp() throws IOException {
|
||||||
@ -1257,7 +1262,7 @@ public void testRackLocalAppReservationThreshold() throws Exception {
|
|||||||
asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
|
asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
|
||||||
|
|
||||||
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
|
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
|
||||||
null, null, null);
|
null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
|
ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
@ -2111,7 +2116,8 @@ public void testQueueDemandCalculation() throws Exception {
|
|||||||
ResourceRequest request1 =
|
ResourceRequest request1 =
|
||||||
createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
|
createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
|
||||||
ask1.add(request1);
|
ask1.add(request1);
|
||||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Second ask, queue2 requests 1 large.
|
// Second ask, queue2 requests 1 large.
|
||||||
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
||||||
@ -2121,7 +2127,8 @@ public void testQueueDemandCalculation() throws Exception {
|
|||||||
ResourceRequest.ANY, 1, 1, false);
|
ResourceRequest.ANY, 1, 1, false);
|
||||||
ask2.add(request2);
|
ask2.add(request2);
|
||||||
ask2.add(request3);
|
ask2.add(request3);
|
||||||
scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Third ask, queue2 requests 2 small (minReqSize).
|
// Third ask, queue2 requests 2 small (minReqSize).
|
||||||
List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
|
||||||
@ -2131,7 +2138,8 @@ public void testQueueDemandCalculation() throws Exception {
|
|||||||
ResourceRequest.ANY, 2, 2, true);
|
ResourceRequest.ANY, 2, 2, true);
|
||||||
ask3.add(request4);
|
ask3.add(request4);
|
||||||
ask3.add(request5);
|
ask3.add(request5);
|
||||||
scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
|
|
||||||
@ -2665,7 +2673,7 @@ public void testReservationWhileMultiplePriorities() throws IOException {
|
|||||||
|
|
||||||
// Complete container
|
// Complete container
|
||||||
scheduler.allocate(attId, new ArrayList<ResourceRequest>(),
|
scheduler.allocate(attId, new ArrayList<ResourceRequest>(),
|
||||||
Arrays.asList(containerId), null, null, null, null);
|
Arrays.asList(containerId), null, null, NULL_UPDATE_REQUESTS);
|
||||||
assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB());
|
assertEquals(1024, scheduler.getRootQueueMetrics().getAvailableMB());
|
||||||
assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
assertEquals(4, scheduler.getRootQueueMetrics().getAvailableVirtualCores());
|
||||||
|
|
||||||
@ -2757,7 +2765,7 @@ public void testMultipleNodesSingleRackRequest() throws Exception {
|
|||||||
asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
|
asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
|
||||||
|
|
||||||
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
|
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
|
||||||
null, null, null);
|
null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// node 1 checks in
|
// node 1 checks in
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
@ -3203,7 +3211,8 @@ public void testCancelStrictLocality() throws IOException {
|
|||||||
createResourceRequest(1024, node1.getHostName(), 1, 0, true),
|
createResourceRequest(1024, node1.getHostName(), 1, 0, true),
|
||||||
createResourceRequest(1024, "rack1", 1, 0, true),
|
createResourceRequest(1024, "rack1", 1, 0, true),
|
||||||
createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
|
createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
|
||||||
scheduler.allocate(attId1, update, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(attId1, update, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// then node2 should get the container
|
// then node2 should get the container
|
||||||
scheduler.handle(node2UpdateEvent);
|
scheduler.handle(node2UpdateEvent);
|
||||||
@ -3250,7 +3259,7 @@ public void testReservationsStrictLocality() throws IOException {
|
|||||||
anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
|
anyRequest = createResourceRequest(1024, ResourceRequest.ANY,
|
||||||
1, 1, false);
|
1, 1, false);
|
||||||
scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest),
|
scheduler.allocate(attId, Arrays.asList(rackRequest, anyRequest),
|
||||||
new ArrayList<ContainerId>(), null, null, null, null);
|
new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
scheduler.handle(nodeUpdateEvent);
|
scheduler.handle(nodeUpdateEvent);
|
||||||
assertEquals(0, app.getReservedContainers().size());
|
assertEquals(0, app.getReservedContainers().size());
|
||||||
@ -4275,7 +4284,7 @@ public void testSchedulingOnRemovedNode() throws Exception {
|
|||||||
|
|
||||||
ask1.add(request1);
|
ask1.add(request1);
|
||||||
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
|
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
|
||||||
null, null, null);
|
null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
String hostName = "127.0.0.1";
|
String hostName = "127.0.0.1";
|
||||||
RMNode node1 = MockNodes.newNodeInfo(1,
|
RMNode node1 = MockNodes.newNodeInfo(1,
|
||||||
@ -4351,11 +4360,11 @@ public void testBlacklistNodes() throws Exception {
|
|||||||
// Verify the blacklist can be updated independent of requesting containers
|
// Verify the blacklist can be updated independent of requesting containers
|
||||||
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
Collections.singletonList(host), null, null, null);
|
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
|
||||||
assertTrue(app.isPlaceBlacklisted(host));
|
assertTrue(app.isPlaceBlacklisted(host));
|
||||||
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(), null,
|
Collections.<ContainerId>emptyList(), null,
|
||||||
Collections.singletonList(host), null, null);
|
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
|
||||||
assertFalse(scheduler.getSchedulerApp(appAttemptId)
|
assertFalse(scheduler.getSchedulerApp(appAttemptId)
|
||||||
.isPlaceBlacklisted(host));
|
.isPlaceBlacklisted(host));
|
||||||
|
|
||||||
@ -4365,7 +4374,7 @@ public void testBlacklistNodes() throws Exception {
|
|||||||
// Verify a container does not actually get placed on the blacklisted host
|
// Verify a container does not actually get placed on the blacklisted host
|
||||||
scheduler.allocate(appAttemptId, update,
|
scheduler.allocate(appAttemptId, update,
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
Collections.singletonList(host), null, null, null);
|
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
|
||||||
assertTrue(app.isPlaceBlacklisted(host));
|
assertTrue(app.isPlaceBlacklisted(host));
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
scheduler.handle(updateEvent);
|
scheduler.handle(updateEvent);
|
||||||
@ -4375,7 +4384,7 @@ public void testBlacklistNodes() throws Exception {
|
|||||||
// Verify a container gets placed on the empty blacklist
|
// Verify a container gets placed on the empty blacklist
|
||||||
scheduler.allocate(appAttemptId, update,
|
scheduler.allocate(appAttemptId, update,
|
||||||
Collections.<ContainerId>emptyList(), null,
|
Collections.<ContainerId>emptyList(), null,
|
||||||
Collections.singletonList(host), null, null);
|
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
|
||||||
assertFalse(app.isPlaceBlacklisted(host));
|
assertFalse(app.isPlaceBlacklisted(host));
|
||||||
createSchedulingRequest(GB, "root.default", "user", 1);
|
createSchedulingRequest(GB, "root.default", "user", 1);
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
|
@ -83,6 +83,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
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.SchedulerAppReport;
|
||||||
@ -119,7 +120,10 @@ public class TestFifoScheduler {
|
|||||||
private static Configuration conf;
|
private static Configuration conf;
|
||||||
private static final RecordFactory recordFactory =
|
private static final RecordFactory recordFactory =
|
||||||
RecordFactoryProvider.getRecordFactory(null);
|
RecordFactoryProvider.getRecordFactory(null);
|
||||||
|
|
||||||
|
private final static ContainerUpdates NULL_UPDATE_REQUESTS =
|
||||||
|
new ContainerUpdates();
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
conf = new Configuration();
|
conf = new Configuration();
|
||||||
@ -274,7 +278,8 @@ public void testNodeLocalAssignment() throws Exception {
|
|||||||
ask.add(nodeLocal);
|
ask.add(nodeLocal);
|
||||||
ask.add(rackLocal);
|
ask.add(rackLocal);
|
||||||
ask.add(any);
|
ask.add(any);
|
||||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
|
NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
|
||||||
|
|
||||||
@ -368,7 +373,8 @@ public void testUpdateResourceOnNode() throws Exception {
|
|||||||
ask.add(nodeLocal);
|
ask.add(nodeLocal);
|
||||||
ask.add(rackLocal);
|
ask.add(rackLocal);
|
||||||
ask.add(any);
|
ask.add(any);
|
||||||
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
|
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Before the node update event, there are one local request
|
// Before the node update event, there are one local request
|
||||||
Assert.assertEquals(1, nodeLocal.getNumContainers());
|
Assert.assertEquals(1, nodeLocal.getNumContainers());
|
||||||
@ -944,7 +950,7 @@ public void testBlackListNodes() throws Exception {
|
|||||||
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
|
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
|
||||||
RMNodeLabelsManager.NO_LABEL));
|
RMNodeLabelsManager.NO_LABEL));
|
||||||
fs.allocate(appAttemptId1, ask1, emptyId,
|
fs.allocate(appAttemptId1, ask1, emptyId,
|
||||||
Collections.singletonList(host_1_0), null, null, null);
|
Collections.singletonList(host_1_0), null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Trigger container assignment
|
// Trigger container assignment
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n3));
|
fs.handle(new NodeUpdateSchedulerEvent(n3));
|
||||||
@ -952,14 +958,16 @@ public void testBlackListNodes() throws Exception {
|
|||||||
// Get the allocation for the application and verify no allocation on
|
// Get the allocation for the application and verify no allocation on
|
||||||
// blacklist node
|
// blacklist node
|
||||||
Allocation allocation1 =
|
Allocation allocation1 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
|
Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
|
||||||
|
|
||||||
// verify host_1_1 can get allocated as not in blacklist
|
// verify host_1_1 can get allocated as not in blacklist
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n4));
|
fs.handle(new NodeUpdateSchedulerEvent(n4));
|
||||||
Allocation allocation2 =
|
Allocation allocation2 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
|
Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
|
||||||
List<Container> containerList = allocation2.getContainers();
|
List<Container> containerList = allocation2.getContainers();
|
||||||
for (Container container : containerList) {
|
for (Container container : containerList) {
|
||||||
@ -974,29 +982,33 @@ public void testBlackListNodes() throws Exception {
|
|||||||
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
||||||
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
|
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
|
||||||
fs.allocate(appAttemptId1, ask2, emptyId,
|
fs.allocate(appAttemptId1, ask2, emptyId,
|
||||||
Collections.singletonList("rack0"), null, null, null);
|
Collections.singletonList("rack0"), null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// verify n1 is not qualified to be allocated
|
// verify n1 is not qualified to be allocated
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n1));
|
fs.handle(new NodeUpdateSchedulerEvent(n1));
|
||||||
Allocation allocation3 =
|
Allocation allocation3 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
|
Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
|
||||||
|
|
||||||
// verify n2 is not qualified to be allocated
|
// verify n2 is not qualified to be allocated
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n2));
|
fs.handle(new NodeUpdateSchedulerEvent(n2));
|
||||||
Allocation allocation4 =
|
Allocation allocation4 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
|
Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
|
||||||
|
|
||||||
// verify n3 is not qualified to be allocated
|
// verify n3 is not qualified to be allocated
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n3));
|
fs.handle(new NodeUpdateSchedulerEvent(n3));
|
||||||
Allocation allocation5 =
|
Allocation allocation5 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
|
Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
|
||||||
|
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n4));
|
fs.handle(new NodeUpdateSchedulerEvent(n4));
|
||||||
Allocation allocation6 =
|
Allocation allocation6 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
|
Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
|
||||||
|
|
||||||
containerList = allocation6.getContainers();
|
containerList = allocation6.getContainers();
|
||||||
@ -1055,25 +1067,29 @@ public void testHeadroom() throws Exception {
|
|||||||
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
|
||||||
ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
||||||
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
|
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
|
||||||
fs.allocate(appAttemptId1, ask1, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, ask1, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Ask for a 2 GB container for app 2
|
// Ask for a 2 GB container for app 2
|
||||||
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
|
||||||
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
|
||||||
ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
|
ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
|
||||||
fs.allocate(appAttemptId2, ask2, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId2, ask2, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
|
|
||||||
// Trigger container assignment
|
// Trigger container assignment
|
||||||
fs.handle(new NodeUpdateSchedulerEvent(n1));
|
fs.handle(new NodeUpdateSchedulerEvent(n1));
|
||||||
|
|
||||||
// Get the allocation for the applications and verify headroom
|
// Get the allocation for the applications and verify headroom
|
||||||
Allocation allocation1 =
|
Allocation allocation1 =
|
||||||
fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId1, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
|
Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
|
||||||
.getResourceLimit().getMemorySize());
|
.getResourceLimit().getMemorySize());
|
||||||
|
|
||||||
Allocation allocation2 =
|
Allocation allocation2 =
|
||||||
fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null, null, null);
|
fs.allocate(appAttemptId2, emptyAsk, emptyId,
|
||||||
|
null, null, NULL_UPDATE_REQUESTS);
|
||||||
Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
|
Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
|
||||||
.getResourceLimit().getMemorySize());
|
.getResourceLimit().getMemorySize());
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user