|
@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
|
|
import org.apache.hadoop.yarn.api.records.PreemptionMessage;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
+import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.Token;
|
|
|
import org.apache.hadoop.yarn.client.api.NMTokenCache;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
@@ -80,6 +81,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.util.Clock;
|
|
|
import org.apache.hadoop.yarn.util.RackResolver;
|
|
|
+import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
@@ -149,8 +151,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
private int lastCompletedTasks = 0;
|
|
|
|
|
|
private boolean recalculateReduceSchedule = false;
|
|
|
- private int mapResourceRequest;//memory
|
|
|
- private int reduceResourceRequest;//memory
|
|
|
+ private Resource mapResourceRequest = Resources.none();
|
|
|
+ private Resource reduceResourceRequest = Resources.none();
|
|
|
|
|
|
private boolean reduceStarted = false;
|
|
|
private float maxReduceRampupLimit = 0;
|
|
@@ -328,49 +330,61 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
|
|
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
|
|
|
JobId jobId = getJob().getID();
|
|
|
- int supportedMaxContainerCapability =
|
|
|
- getMaxContainerCapability().getMemory();
|
|
|
+ Resource supportedMaxContainerCapability = getMaxContainerCapability();
|
|
|
if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
|
|
|
- if (mapResourceRequest == 0) {
|
|
|
- mapResourceRequest = reqEvent.getCapability().getMemory();
|
|
|
- eventHandler.handle(new JobHistoryEvent(jobId,
|
|
|
- new NormalizedResourceEvent(org.apache.hadoop.mapreduce.TaskType.MAP,
|
|
|
- mapResourceRequest)));
|
|
|
- LOG.info("mapResourceRequest:"+ mapResourceRequest);
|
|
|
- if (mapResourceRequest > supportedMaxContainerCapability) {
|
|
|
- String diagMsg = "MAP capability required is more than the supported " +
|
|
|
- "max container capability in the cluster. Killing the Job. mapResourceRequest: " +
|
|
|
- mapResourceRequest + " maxContainerCapability:" + supportedMaxContainerCapability;
|
|
|
+ if (mapResourceRequest.equals(Resources.none())) {
|
|
|
+ mapResourceRequest = reqEvent.getCapability();
|
|
|
+ eventHandler.handle(new JobHistoryEvent(jobId,
|
|
|
+ new NormalizedResourceEvent(
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.MAP, mapResourceRequest
|
|
|
+ .getMemory())));
|
|
|
+ LOG.info("mapResourceRequest:" + mapResourceRequest);
|
|
|
+ if (mapResourceRequest.getMemory() > supportedMaxContainerCapability
|
|
|
+ .getMemory()
|
|
|
+ || mapResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
|
|
+ .getVirtualCores()) {
|
|
|
+ String diagMsg =
|
|
|
+ "MAP capability required is more than the supported "
|
|
|
+ + "max container capability in the cluster. Killing the Job. mapResourceRequest: "
|
|
|
+ + mapResourceRequest + " maxContainerCapability:"
|
|
|
+ + supportedMaxContainerCapability;
|
|
|
LOG.info(diagMsg);
|
|
|
- eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
|
|
- jobId, diagMsg));
|
|
|
+ eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
|
|
|
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
}
|
|
|
}
|
|
|
- //set the rounded off memory
|
|
|
- reqEvent.getCapability().setMemory(mapResourceRequest);
|
|
|
+ // set the resources
|
|
|
+ reqEvent.getCapability().setMemory(mapResourceRequest.getMemory());
|
|
|
+ reqEvent.getCapability().setVirtualCores(
|
|
|
+ mapResourceRequest.getVirtualCores());
|
|
|
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
|
|
|
} else {
|
|
|
- if (reduceResourceRequest == 0) {
|
|
|
- reduceResourceRequest = reqEvent.getCapability().getMemory();
|
|
|
- eventHandler.handle(new JobHistoryEvent(jobId,
|
|
|
- new NormalizedResourceEvent(
|
|
|
- org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
|
|
- reduceResourceRequest)));
|
|
|
- LOG.info("reduceResourceRequest:"+ reduceResourceRequest);
|
|
|
- if (reduceResourceRequest > supportedMaxContainerCapability) {
|
|
|
- String diagMsg = "REDUCE capability required is more than the " +
|
|
|
- "supported max container capability in the cluster. Killing the " +
|
|
|
- "Job. reduceResourceRequest: " + reduceResourceRequest +
|
|
|
- " maxContainerCapability:" + supportedMaxContainerCapability;
|
|
|
+ if (reduceResourceRequest.equals(Resources.none())) {
|
|
|
+ reduceResourceRequest = reqEvent.getCapability();
|
|
|
+ eventHandler.handle(new JobHistoryEvent(jobId,
|
|
|
+ new NormalizedResourceEvent(
|
|
|
+ org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
|
|
+ reduceResourceRequest.getMemory())));
|
|
|
+ LOG.info("reduceResourceRequest:" + reduceResourceRequest);
|
|
|
+ if (reduceResourceRequest.getMemory() > supportedMaxContainerCapability
|
|
|
+ .getMemory()
|
|
|
+ || reduceResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
|
|
+ .getVirtualCores()) {
|
|
|
+ String diagMsg =
|
|
|
+ "REDUCE capability required is more than the "
|
|
|
+ + "supported max container capability in the cluster. Killing the "
|
|
|
+ + "Job. reduceResourceRequest: " + reduceResourceRequest
|
|
|
+ + " maxContainerCapability:"
|
|
|
+ + supportedMaxContainerCapability;
|
|
|
LOG.info(diagMsg);
|
|
|
- eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
|
|
- jobId, diagMsg));
|
|
|
+ eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
|
|
|
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
|
|
}
|
|
|
}
|
|
|
- //set the rounded off memory
|
|
|
- reqEvent.getCapability().setMemory(reduceResourceRequest);
|
|
|
+ // set the resources
|
|
|
+ reqEvent.getCapability().setMemory(reduceResourceRequest.getMemory());
|
|
|
+ reqEvent.getCapability().setVirtualCores(
|
|
|
+ reduceResourceRequest.getVirtualCores());
|
|
|
if (reqEvent.getEarlierAttemptFailed()) {
|
|
|
//add to the front of queue for fail fast
|
|
|
pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
|
|
@@ -425,34 +439,40 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
- synchronized void setReduceResourceRequest(int mem) {
|
|
|
- this.reduceResourceRequest = mem;
|
|
|
+ synchronized void setReduceResourceRequest(Resource res) {
|
|
|
+ this.reduceResourceRequest = res;
|
|
|
}
|
|
|
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
- synchronized void setMapResourceRequest(int mem) {
|
|
|
- this.mapResourceRequest = mem;
|
|
|
+ synchronized void setMapResourceRequest(Resource res) {
|
|
|
+ this.mapResourceRequest = res;
|
|
|
}
|
|
|
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
|
void preemptReducesIfNeeded() {
|
|
|
- if (reduceResourceRequest == 0) {
|
|
|
- return; //no reduces
|
|
|
+ if (reduceResourceRequest.equals(Resources.none())) {
|
|
|
+ return; // no reduces
|
|
|
}
|
|
|
//check if reduces have taken over the whole cluster and there are
|
|
|
//unassigned maps
|
|
|
if (scheduledRequests.maps.size() > 0) {
|
|
|
- int memLimit = getMemLimit();
|
|
|
- int availableMemForMap = memLimit - ((assignedRequests.reduces.size() -
|
|
|
- assignedRequests.preemptionWaitingReduces.size()) * reduceResourceRequest);
|
|
|
- //availableMemForMap must be sufficient to run atleast 1 map
|
|
|
- if (availableMemForMap < mapResourceRequest) {
|
|
|
- //to make sure new containers are given to maps and not reduces
|
|
|
- //ramp down all scheduled reduces if any
|
|
|
- //(since reduces are scheduled at higher priority than maps)
|
|
|
- LOG.info("Ramping down all scheduled reduces:" + scheduledRequests.reduces.size());
|
|
|
+ Resource resourceLimit = getResourceLimit();
|
|
|
+ Resource availableResourceForMap =
|
|
|
+ Resources.subtract(
|
|
|
+ resourceLimit,
|
|
|
+ Resources.multiply(reduceResourceRequest,
|
|
|
+ assignedRequests.reduces.size()
|
|
|
+ - assignedRequests.preemptionWaitingReduces.size()));
|
|
|
+ // availableMemForMap must be sufficient to run at least 1 map
|
|
|
+ if (ResourceCalculatorUtils.computeAvailableContainers(availableResourceForMap,
|
|
|
+ mapResourceRequest, getSchedulerResourceTypes()) <= 0) {
|
|
|
+ // to make sure new containers are given to maps and not reduces
|
|
|
+ // ramp down all scheduled reduces if any
|
|
|
+ // (since reduces are scheduled at higher priority than maps)
|
|
|
+ LOG.info("Ramping down all scheduled reduces:"
|
|
|
+ + scheduledRequests.reduces.size());
|
|
|
for (ContainerRequest req : scheduledRequests.reduces.values()) {
|
|
|
pendingReduces.add(req);
|
|
|
}
|
|
@@ -462,17 +482,25 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
//hanging around for a while
|
|
|
int hangingMapRequests = getNumOfHangingRequests(scheduledRequests.maps);
|
|
|
if (hangingMapRequests > 0) {
|
|
|
- //preempt for making space for at least one map
|
|
|
- int premeptionLimit = Math.max(mapResourceRequest,
|
|
|
- (int) (maxReducePreemptionLimit * memLimit));
|
|
|
-
|
|
|
- int preemptMem = Math.min(hangingMapRequests * mapResourceRequest,
|
|
|
- premeptionLimit);
|
|
|
-
|
|
|
- int toPreempt = (int) Math.ceil((float) preemptMem / reduceResourceRequest);
|
|
|
- toPreempt = Math.min(toPreempt, assignedRequests.reduces.size());
|
|
|
-
|
|
|
- LOG.info("Going to preempt " + toPreempt + " due to lack of space for maps");
|
|
|
+ // preempt for making space for at least one map
|
|
|
+ int preemptionReduceNumForOneMap =
|
|
|
+ ResourceCalculatorUtils.divideAndCeilContainers(mapResourceRequest,
|
|
|
+ reduceResourceRequest, getSchedulerResourceTypes());
|
|
|
+ int preemptionReduceNumForPreemptionLimit =
|
|
|
+ ResourceCalculatorUtils.divideAndCeilContainers(
|
|
|
+ Resources.multiply(resourceLimit, maxReducePreemptionLimit),
|
|
|
+ reduceResourceRequest, getSchedulerResourceTypes());
|
|
|
+ int preemptionReduceNumForAllMaps =
|
|
|
+ ResourceCalculatorUtils.divideAndCeilContainers(
|
|
|
+ Resources.multiply(mapResourceRequest, hangingMapRequests),
|
|
|
+ reduceResourceRequest, getSchedulerResourceTypes());
|
|
|
+ int toPreempt =
|
|
|
+ Math.min(Math.max(preemptionReduceNumForOneMap,
|
|
|
+ preemptionReduceNumForPreemptionLimit),
|
|
|
+ preemptionReduceNumForAllMaps);
|
|
|
+
|
|
|
+ LOG.info("Going to preempt " + toPreempt
|
|
|
+ + " due to lack of space for maps");
|
|
|
assignedRequests.preemptReduce(toPreempt);
|
|
|
}
|
|
|
}
|
|
@@ -497,7 +525,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
int totalMaps, int completedMaps,
|
|
|
int scheduledMaps, int scheduledReduces,
|
|
|
int assignedMaps, int assignedReduces,
|
|
|
- int mapResourceReqt, int reduceResourceReqt,
|
|
|
+ Resource mapResourceReqt, Resource reduceResourceReqt,
|
|
|
int numPendingReduces,
|
|
|
float maxReduceRampupLimit, float reduceSlowStart) {
|
|
|
|
|
@@ -505,8 +533,12 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- int headRoom = getAvailableResources() != null ?
|
|
|
- getAvailableResources().getMemory() : 0;
|
|
|
+ // get available resources for this job
|
|
|
+ Resource headRoom = getAvailableResources();
|
|
|
+ if (headRoom == null) {
|
|
|
+ headRoom = Resources.none();
|
|
|
+ }
|
|
|
+
|
|
|
LOG.info("Recalculating schedule, headroom=" + headRoom);
|
|
|
|
|
|
//check for slow start
|
|
@@ -540,49 +572,60 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
completedMapPercent = 1;
|
|
|
}
|
|
|
|
|
|
- int netScheduledMapMem =
|
|
|
- (scheduledMaps + assignedMaps) * mapResourceReqt;
|
|
|
+ Resource netScheduledMapResource =
|
|
|
+ Resources.multiply(mapResourceReqt, (scheduledMaps + assignedMaps));
|
|
|
|
|
|
- int netScheduledReduceMem =
|
|
|
- (scheduledReduces + assignedReduces) * reduceResourceReqt;
|
|
|
+ Resource netScheduledReduceResource =
|
|
|
+ Resources.multiply(reduceResourceReqt,
|
|
|
+ (scheduledReduces + assignedReduces));
|
|
|
+
|
|
|
+ Resource finalMapResourceLimit;
|
|
|
+ Resource finalReduceResourceLimit;
|
|
|
|
|
|
- int finalMapMemLimit = 0;
|
|
|
- int finalReduceMemLimit = 0;
|
|
|
-
|
|
|
// ramp up the reduces based on completed map percentage
|
|
|
- int totalMemLimit = getMemLimit();
|
|
|
- int idealReduceMemLimit =
|
|
|
- Math.min(
|
|
|
- (int)(completedMapPercent * totalMemLimit),
|
|
|
- (int) (maxReduceRampupLimit * totalMemLimit));
|
|
|
- int idealMapMemLimit = totalMemLimit - idealReduceMemLimit;
|
|
|
+ Resource totalResourceLimit = getResourceLimit();
|
|
|
+
|
|
|
+ Resource idealReduceResourceLimit =
|
|
|
+ Resources.multiply(totalResourceLimit,
|
|
|
+ Math.min(completedMapPercent, maxReduceRampupLimit));
|
|
|
+ Resource ideaMapResourceLimit =
|
|
|
+ Resources.subtract(totalResourceLimit, idealReduceResourceLimit);
|
|
|
|
|
|
// check if there aren't enough maps scheduled, give the free map capacity
|
|
|
- // to reduce
|
|
|
- if (idealMapMemLimit > netScheduledMapMem) {
|
|
|
- int unusedMapMemLimit = idealMapMemLimit - netScheduledMapMem;
|
|
|
- finalReduceMemLimit = idealReduceMemLimit + unusedMapMemLimit;
|
|
|
- finalMapMemLimit = totalMemLimit - finalReduceMemLimit;
|
|
|
+ // to reduce.
|
|
|
+ // Even when container number equals, there may be unused resources in one
|
|
|
+ // dimension
|
|
|
+ if (ResourceCalculatorUtils.computeAvailableContainers(ideaMapResourceLimit,
|
|
|
+ mapResourceReqt, getSchedulerResourceTypes()) >= (scheduledMaps + assignedMaps)) {
|
|
|
+ // enough resource given to maps, given the remaining to reduces
|
|
|
+ Resource unusedMapResourceLimit =
|
|
|
+ Resources.subtract(ideaMapResourceLimit, netScheduledMapResource);
|
|
|
+ finalReduceResourceLimit =
|
|
|
+ Resources.add(idealReduceResourceLimit, unusedMapResourceLimit);
|
|
|
+ finalMapResourceLimit =
|
|
|
+ Resources.subtract(totalResourceLimit, finalReduceResourceLimit);
|
|
|
} else {
|
|
|
- finalMapMemLimit = idealMapMemLimit;
|
|
|
- finalReduceMemLimit = idealReduceMemLimit;
|
|
|
+ finalMapResourceLimit = ideaMapResourceLimit;
|
|
|
+ finalReduceResourceLimit = idealReduceResourceLimit;
|
|
|
}
|
|
|
-
|
|
|
- LOG.info("completedMapPercent " + completedMapPercent +
|
|
|
- " totalMemLimit:" + totalMemLimit +
|
|
|
- " finalMapMemLimit:" + finalMapMemLimit +
|
|
|
- " finalReduceMemLimit:" + finalReduceMemLimit +
|
|
|
- " netScheduledMapMem:" + netScheduledMapMem +
|
|
|
- " netScheduledReduceMem:" + netScheduledReduceMem);
|
|
|
-
|
|
|
- int rampUp =
|
|
|
- (finalReduceMemLimit - netScheduledReduceMem) / reduceResourceReqt;
|
|
|
-
|
|
|
+
|
|
|
+ LOG.info("completedMapPercent " + completedMapPercent
|
|
|
+ + " totalResourceLimit:" + totalResourceLimit
|
|
|
+ + " finalMapResourceLimit:" + finalMapResourceLimit
|
|
|
+ + " finalReduceResourceLimit:" + finalReduceResourceLimit
|
|
|
+ + " netScheduledMapResource:" + netScheduledMapResource
|
|
|
+ + " netScheduledReduceResource:" + netScheduledReduceResource);
|
|
|
+
|
|
|
+ int rampUp =
|
|
|
+ ResourceCalculatorUtils.computeAvailableContainers(Resources.subtract(
|
|
|
+ finalReduceResourceLimit, netScheduledReduceResource),
|
|
|
+ reduceResourceReqt, getSchedulerResourceTypes());
|
|
|
+
|
|
|
if (rampUp > 0) {
|
|
|
rampUp = Math.min(rampUp, numPendingReduces);
|
|
|
LOG.info("Ramping up " + rampUp);
|
|
|
rampUpReduces(rampUp);
|
|
|
- } else if (rampUp < 0){
|
|
|
+ } else if (rampUp < 0) {
|
|
|
int rampDown = -1 * rampUp;
|
|
|
rampDown = Math.min(rampDown, scheduledReduces);
|
|
|
LOG.info("Ramping down " + rampDown);
|
|
@@ -618,8 +661,10 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
private List<Container> getResources() throws Exception {
|
|
|
- int headRoom = getAvailableResources() != null
|
|
|
- ? getAvailableResources().getMemory() : 0;//first time it would be null
|
|
|
+ // will be null the first time
|
|
|
+ Resource headRoom =
|
|
|
+ getAvailableResources() == null ? Resources.none() :
|
|
|
+ Resources.clone(getAvailableResources());
|
|
|
AllocateResponse response;
|
|
|
/*
|
|
|
* If contact with RM is lost, the AM will wait MR_AM_TO_RM_WAIT_INTERVAL_MS
|
|
@@ -670,7 +715,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
throw new YarnRuntimeException(msg);
|
|
|
}
|
|
|
}
|
|
|
- int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
|
|
|
+ Resource newHeadRoom =
|
|
|
+ getAvailableResources() == null ? Resources.none()
|
|
|
+ : getAvailableResources();
|
|
|
List<Container> newContainers = response.getAllocatedContainers();
|
|
|
// Setting NMTokens
|
|
|
if (response.getNMTokens() != null) {
|
|
@@ -694,10 +741,11 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
new PreemptionContext(assignedRequests), preemptReq);
|
|
|
}
|
|
|
|
|
|
- if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
|
|
|
+ if (newContainers.size() + finishedContainers.size() > 0
|
|
|
+ || !headRoom.equals(newHeadRoom)) {
|
|
|
//something changed
|
|
|
recalculateReduceSchedule = true;
|
|
|
- if (LOG.isDebugEnabled() && headRoom != newHeadRoom) {
|
|
|
+ if (LOG.isDebugEnabled() && !headRoom.equals(newHeadRoom)) {
|
|
|
LOG.debug("headroom=" + newHeadRoom);
|
|
|
}
|
|
|
}
|
|
@@ -802,10 +850,18 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
}
|
|
|
|
|
|
@Private
|
|
|
- public int getMemLimit() {
|
|
|
- int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
|
|
|
- return headRoom + assignedRequests.maps.size() * mapResourceRequest +
|
|
|
- assignedRequests.reduces.size() * reduceResourceRequest;
|
|
|
+ public Resource getResourceLimit() {
|
|
|
+ Resource headRoom = getAvailableResources();
|
|
|
+ if (headRoom == null) {
|
|
|
+ headRoom = Resources.none();
|
|
|
+ }
|
|
|
+ Resource assignedMapResource =
|
|
|
+ Resources.multiply(mapResourceRequest, assignedRequests.maps.size());
|
|
|
+ Resource assignedReduceResource =
|
|
|
+ Resources.multiply(reduceResourceRequest,
|
|
|
+ assignedRequests.reduces.size());
|
|
|
+ return Resources.add(headRoom,
|
|
|
+ Resources.add(assignedMapResource, assignedReduceResource));
|
|
|
}
|
|
|
|
|
|
@Private
|
|
@@ -914,10 +970,11 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
// a container to be assigned
|
|
|
boolean isAssignable = true;
|
|
|
Priority priority = allocated.getPriority();
|
|
|
- int allocatedMemory = allocated.getResource().getMemory();
|
|
|
+ Resource allocatedResource = allocated.getResource();
|
|
|
if (PRIORITY_FAST_FAIL_MAP.equals(priority)
|
|
|
|| PRIORITY_MAP.equals(priority)) {
|
|
|
- if (allocatedMemory < mapResourceRequest
|
|
|
+ if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
|
|
|
+ mapResourceRequest, getSchedulerResourceTypes()) <= 0
|
|
|
|| maps.isEmpty()) {
|
|
|
LOG.info("Cannot assign container " + allocated
|
|
|
+ " for a map as either "
|
|
@@ -928,7 +985,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
}
|
|
|
}
|
|
|
else if (PRIORITY_REDUCE.equals(priority)) {
|
|
|
- if (allocatedMemory < reduceResourceRequest
|
|
|
+ if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
|
|
|
+ reduceResourceRequest, getSchedulerResourceTypes()) <= 0
|
|
|
|| reduces.isEmpty()) {
|
|
|
LOG.info("Cannot assign container " + allocated
|
|
|
+ " for a reduce as either "
|