MAPREDUCE-5279. Made MR headroom calculation honor cpu dimension when YARN scheduler resource type is memory plus cpu. Contributed by Peng Zhang and Varun Vasudev.
This commit is contained in:
parent
26cba7f35f
commit
376233cdd4
|
@ -254,6 +254,10 @@ Release 2.6.0 - UNRELEASED
|
||||||
MAPREDUCE-5891. Improved shuffle error handling across NM restarts
|
MAPREDUCE-5891. Improved shuffle error handling across NM restarts
|
||||||
(Junping Du via jlowe)
|
(Junping Du via jlowe)
|
||||||
|
|
||||||
|
MAPREDUCE-5279. Made MR headroom calculation honor cpu dimension when YARN
|
||||||
|
scheduler resource type is memory plus cpu. (Peng Zhang and Varun Vasudev
|
||||||
|
via zjshen)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.EnumSet;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
@ -59,6 +60,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Registers/unregisters to RM and sends heartbeats to RM.
|
* Registers/unregisters to RM and sends heartbeats to RM.
|
||||||
|
@ -90,6 +92,8 @@ public abstract class RMCommunicator extends AbstractService
|
||||||
private volatile boolean shouldUnregister = true;
|
private volatile boolean shouldUnregister = true;
|
||||||
private boolean isApplicationMasterRegistered = false;
|
private boolean isApplicationMasterRegistered = false;
|
||||||
|
|
||||||
|
private EnumSet<SchedulerResourceTypes> schedulerResourceTypes;
|
||||||
|
|
||||||
public RMCommunicator(ClientService clientService, AppContext context) {
|
public RMCommunicator(ClientService clientService, AppContext context) {
|
||||||
super("RMCommunicator");
|
super("RMCommunicator");
|
||||||
this.clientService = clientService;
|
this.clientService = clientService;
|
||||||
|
@ -98,6 +102,7 @@ public abstract class RMCommunicator extends AbstractService
|
||||||
this.applicationId = context.getApplicationID();
|
this.applicationId = context.getApplicationID();
|
||||||
this.stopped = new AtomicBoolean(false);
|
this.stopped = new AtomicBoolean(false);
|
||||||
this.heartbeatCallbacks = new ConcurrentLinkedQueue<Runnable>();
|
this.heartbeatCallbacks = new ConcurrentLinkedQueue<Runnable>();
|
||||||
|
this.schedulerResourceTypes = EnumSet.of(SchedulerResourceTypes.MEMORY);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -163,10 +168,11 @@ public abstract class RMCommunicator extends AbstractService
|
||||||
setClientToAMToken(response.getClientToAMTokenMasterKey());
|
setClientToAMToken(response.getClientToAMTokenMasterKey());
|
||||||
}
|
}
|
||||||
this.applicationACLs = response.getApplicationACLs();
|
this.applicationACLs = response.getApplicationACLs();
|
||||||
LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
|
LOG.info("maxContainerCapability: " + maxContainerCapability);
|
||||||
String queue = response.getQueue();
|
String queue = response.getQueue();
|
||||||
LOG.info("queue: " + queue);
|
LOG.info("queue: " + queue);
|
||||||
job.setQueueName(queue);
|
job.setQueueName(queue);
|
||||||
|
this.schedulerResourceTypes.addAll(response.getSchedulerResourceTypes());
|
||||||
} catch (Exception are) {
|
} catch (Exception are) {
|
||||||
LOG.error("Exception while registering", are);
|
LOG.error("Exception while registering", are);
|
||||||
throw new YarnRuntimeException(are);
|
throw new YarnRuntimeException(are);
|
||||||
|
@ -343,4 +349,8 @@ public abstract class RMCommunicator extends AbstractService
|
||||||
protected boolean isApplicationMasterRegistered() {
|
protected boolean isApplicationMasterRegistered() {
|
||||||
return isApplicationMasterRegistered;
|
return isApplicationMasterRegistered;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public EnumSet<SchedulerResourceTypes> getSchedulerResourceTypes() {
|
||||||
|
return schedulerResourceTypes;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.NodeState;
|
||||||
import org.apache.hadoop.yarn.api.records.PreemptionMessage;
|
import org.apache.hadoop.yarn.api.records.PreemptionMessage;
|
||||||
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.Token;
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
import org.apache.hadoop.yarn.client.api.NMTokenCache;
|
import org.apache.hadoop.yarn.client.api.NMTokenCache;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
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.security.AMRMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.util.Clock;
|
import org.apache.hadoop.yarn.util.Clock;
|
||||||
import org.apache.hadoop.yarn.util.RackResolver;
|
import org.apache.hadoop.yarn.util.RackResolver;
|
||||||
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
@ -149,8 +151,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
private int lastCompletedTasks = 0;
|
private int lastCompletedTasks = 0;
|
||||||
|
|
||||||
private boolean recalculateReduceSchedule = false;
|
private boolean recalculateReduceSchedule = false;
|
||||||
private int mapResourceRequest;//memory
|
private Resource mapResourceRequest = Resources.none();
|
||||||
private int reduceResourceRequest;//memory
|
private Resource reduceResourceRequest = Resources.none();
|
||||||
|
|
||||||
private boolean reduceStarted = false;
|
private boolean reduceStarted = false;
|
||||||
private float maxReduceRampupLimit = 0;
|
private float maxReduceRampupLimit = 0;
|
||||||
|
@ -328,49 +330,61 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
||||||
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
|
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
|
||||||
JobId jobId = getJob().getID();
|
JobId jobId = getJob().getID();
|
||||||
int supportedMaxContainerCapability =
|
Resource supportedMaxContainerCapability = getMaxContainerCapability();
|
||||||
getMaxContainerCapability().getMemory();
|
|
||||||
if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
|
if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
|
||||||
if (mapResourceRequest == 0) {
|
if (mapResourceRequest.equals(Resources.none())) {
|
||||||
mapResourceRequest = reqEvent.getCapability().getMemory();
|
mapResourceRequest = reqEvent.getCapability();
|
||||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||||
new NormalizedResourceEvent(org.apache.hadoop.mapreduce.TaskType.MAP,
|
new NormalizedResourceEvent(
|
||||||
mapResourceRequest)));
|
org.apache.hadoop.mapreduce.TaskType.MAP, mapResourceRequest
|
||||||
LOG.info("mapResourceRequest:"+ mapResourceRequest);
|
.getMemory())));
|
||||||
if (mapResourceRequest > supportedMaxContainerCapability) {
|
LOG.info("mapResourceRequest:" + mapResourceRequest);
|
||||||
String diagMsg = "MAP capability required is more than the supported " +
|
if (mapResourceRequest.getMemory() > supportedMaxContainerCapability
|
||||||
"max container capability in the cluster. Killing the Job. mapResourceRequest: " +
|
.getMemory()
|
||||||
mapResourceRequest + " maxContainerCapability:" + supportedMaxContainerCapability;
|
|| 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);
|
LOG.info(diagMsg);
|
||||||
eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
|
||||||
jobId, diagMsg));
|
|
||||||
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//set the rounded off memory
|
// set the resources
|
||||||
reqEvent.getCapability().setMemory(mapResourceRequest);
|
reqEvent.getCapability().setMemory(mapResourceRequest.getMemory());
|
||||||
|
reqEvent.getCapability().setVirtualCores(
|
||||||
|
mapResourceRequest.getVirtualCores());
|
||||||
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
|
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
|
||||||
} else {
|
} else {
|
||||||
if (reduceResourceRequest == 0) {
|
if (reduceResourceRequest.equals(Resources.none())) {
|
||||||
reduceResourceRequest = reqEvent.getCapability().getMemory();
|
reduceResourceRequest = reqEvent.getCapability();
|
||||||
eventHandler.handle(new JobHistoryEvent(jobId,
|
eventHandler.handle(new JobHistoryEvent(jobId,
|
||||||
new NormalizedResourceEvent(
|
new NormalizedResourceEvent(
|
||||||
org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
org.apache.hadoop.mapreduce.TaskType.REDUCE,
|
||||||
reduceResourceRequest)));
|
reduceResourceRequest.getMemory())));
|
||||||
LOG.info("reduceResourceRequest:"+ reduceResourceRequest);
|
LOG.info("reduceResourceRequest:" + reduceResourceRequest);
|
||||||
if (reduceResourceRequest > supportedMaxContainerCapability) {
|
if (reduceResourceRequest.getMemory() > supportedMaxContainerCapability
|
||||||
String diagMsg = "REDUCE capability required is more than the " +
|
.getMemory()
|
||||||
"supported max container capability in the cluster. Killing the " +
|
|| reduceResourceRequest.getVirtualCores() > supportedMaxContainerCapability
|
||||||
"Job. reduceResourceRequest: " + reduceResourceRequest +
|
.getVirtualCores()) {
|
||||||
" maxContainerCapability:" + supportedMaxContainerCapability;
|
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);
|
LOG.info(diagMsg);
|
||||||
eventHandler.handle(new JobDiagnosticsUpdateEvent(
|
eventHandler.handle(new JobDiagnosticsUpdateEvent(jobId, diagMsg));
|
||||||
jobId, diagMsg));
|
|
||||||
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
eventHandler.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//set the rounded off memory
|
// set the resources
|
||||||
reqEvent.getCapability().setMemory(reduceResourceRequest);
|
reqEvent.getCapability().setMemory(reduceResourceRequest.getMemory());
|
||||||
|
reqEvent.getCapability().setVirtualCores(
|
||||||
|
reduceResourceRequest.getVirtualCores());
|
||||||
if (reqEvent.getEarlierAttemptFailed()) {
|
if (reqEvent.getEarlierAttemptFailed()) {
|
||||||
//add to the front of queue for fail fast
|
//add to the front of queue for fail fast
|
||||||
pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
|
pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
|
||||||
|
@ -425,34 +439,40 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
synchronized void setReduceResourceRequest(int mem) {
|
synchronized void setReduceResourceRequest(Resource res) {
|
||||||
this.reduceResourceRequest = mem;
|
this.reduceResourceRequest = res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
synchronized void setMapResourceRequest(int mem) {
|
synchronized void setMapResourceRequest(Resource res) {
|
||||||
this.mapResourceRequest = mem;
|
this.mapResourceRequest = res;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
void preemptReducesIfNeeded() {
|
void preemptReducesIfNeeded() {
|
||||||
if (reduceResourceRequest == 0) {
|
if (reduceResourceRequest.equals(Resources.none())) {
|
||||||
return; //no reduces
|
return; // no reduces
|
||||||
}
|
}
|
||||||
//check if reduces have taken over the whole cluster and there are
|
//check if reduces have taken over the whole cluster and there are
|
||||||
//unassigned maps
|
//unassigned maps
|
||||||
if (scheduledRequests.maps.size() > 0) {
|
if (scheduledRequests.maps.size() > 0) {
|
||||||
int memLimit = getMemLimit();
|
Resource resourceLimit = getResourceLimit();
|
||||||
int availableMemForMap = memLimit - ((assignedRequests.reduces.size() -
|
Resource availableResourceForMap =
|
||||||
assignedRequests.preemptionWaitingReduces.size()) * reduceResourceRequest);
|
Resources.subtract(
|
||||||
//availableMemForMap must be sufficient to run atleast 1 map
|
resourceLimit,
|
||||||
if (availableMemForMap < mapResourceRequest) {
|
Resources.multiply(reduceResourceRequest,
|
||||||
//to make sure new containers are given to maps and not reduces
|
assignedRequests.reduces.size()
|
||||||
//ramp down all scheduled reduces if any
|
- assignedRequests.preemptionWaitingReduces.size()));
|
||||||
//(since reduces are scheduled at higher priority than maps)
|
// availableMemForMap must be sufficient to run at least 1 map
|
||||||
LOG.info("Ramping down all scheduled reduces:" + scheduledRequests.reduces.size());
|
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()) {
|
for (ContainerRequest req : scheduledRequests.reduces.values()) {
|
||||||
pendingReduces.add(req);
|
pendingReduces.add(req);
|
||||||
}
|
}
|
||||||
|
@ -462,17 +482,25 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
//hanging around for a while
|
//hanging around for a while
|
||||||
int hangingMapRequests = getNumOfHangingRequests(scheduledRequests.maps);
|
int hangingMapRequests = getNumOfHangingRequests(scheduledRequests.maps);
|
||||||
if (hangingMapRequests > 0) {
|
if (hangingMapRequests > 0) {
|
||||||
//preempt for making space for at least one map
|
// preempt for making space for at least one map
|
||||||
int premeptionLimit = Math.max(mapResourceRequest,
|
int preemptionReduceNumForOneMap =
|
||||||
(int) (maxReducePreemptionLimit * memLimit));
|
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);
|
||||||
|
|
||||||
int preemptMem = Math.min(hangingMapRequests * mapResourceRequest,
|
LOG.info("Going to preempt " + toPreempt
|
||||||
premeptionLimit);
|
+ " due to lack of space for maps");
|
||||||
|
|
||||||
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");
|
|
||||||
assignedRequests.preemptReduce(toPreempt);
|
assignedRequests.preemptReduce(toPreempt);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -497,7 +525,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
int totalMaps, int completedMaps,
|
int totalMaps, int completedMaps,
|
||||||
int scheduledMaps, int scheduledReduces,
|
int scheduledMaps, int scheduledReduces,
|
||||||
int assignedMaps, int assignedReduces,
|
int assignedMaps, int assignedReduces,
|
||||||
int mapResourceReqt, int reduceResourceReqt,
|
Resource mapResourceReqt, Resource reduceResourceReqt,
|
||||||
int numPendingReduces,
|
int numPendingReduces,
|
||||||
float maxReduceRampupLimit, float reduceSlowStart) {
|
float maxReduceRampupLimit, float reduceSlowStart) {
|
||||||
|
|
||||||
|
@ -505,8 +533,12 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
int headRoom = getAvailableResources() != null ?
|
// get available resources for this job
|
||||||
getAvailableResources().getMemory() : 0;
|
Resource headRoom = getAvailableResources();
|
||||||
|
if (headRoom == null) {
|
||||||
|
headRoom = Resources.none();
|
||||||
|
}
|
||||||
|
|
||||||
LOG.info("Recalculating schedule, headroom=" + headRoom);
|
LOG.info("Recalculating schedule, headroom=" + headRoom);
|
||||||
|
|
||||||
//check for slow start
|
//check for slow start
|
||||||
|
@ -540,49 +572,60 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
completedMapPercent = 1;
|
completedMapPercent = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
int netScheduledMapMem =
|
Resource netScheduledMapResource =
|
||||||
(scheduledMaps + assignedMaps) * mapResourceReqt;
|
Resources.multiply(mapResourceReqt, (scheduledMaps + assignedMaps));
|
||||||
|
|
||||||
int netScheduledReduceMem =
|
Resource netScheduledReduceResource =
|
||||||
(scheduledReduces + assignedReduces) * reduceResourceReqt;
|
Resources.multiply(reduceResourceReqt,
|
||||||
|
(scheduledReduces + assignedReduces));
|
||||||
|
|
||||||
|
Resource finalMapResourceLimit;
|
||||||
|
Resource finalReduceResourceLimit;
|
||||||
|
|
||||||
int finalMapMemLimit = 0;
|
|
||||||
int finalReduceMemLimit = 0;
|
|
||||||
|
|
||||||
// ramp up the reduces based on completed map percentage
|
// ramp up the reduces based on completed map percentage
|
||||||
int totalMemLimit = getMemLimit();
|
Resource totalResourceLimit = getResourceLimit();
|
||||||
int idealReduceMemLimit =
|
|
||||||
Math.min(
|
Resource idealReduceResourceLimit =
|
||||||
(int)(completedMapPercent * totalMemLimit),
|
Resources.multiply(totalResourceLimit,
|
||||||
(int) (maxReduceRampupLimit * totalMemLimit));
|
Math.min(completedMapPercent, maxReduceRampupLimit));
|
||||||
int idealMapMemLimit = totalMemLimit - idealReduceMemLimit;
|
Resource ideaMapResourceLimit =
|
||||||
|
Resources.subtract(totalResourceLimit, idealReduceResourceLimit);
|
||||||
|
|
||||||
// check if there aren't enough maps scheduled, give the free map capacity
|
// check if there aren't enough maps scheduled, give the free map capacity
|
||||||
// to reduce
|
// to reduce.
|
||||||
if (idealMapMemLimit > netScheduledMapMem) {
|
// Even when container number equals, there may be unused resources in one
|
||||||
int unusedMapMemLimit = idealMapMemLimit - netScheduledMapMem;
|
// dimension
|
||||||
finalReduceMemLimit = idealReduceMemLimit + unusedMapMemLimit;
|
if (ResourceCalculatorUtils.computeAvailableContainers(ideaMapResourceLimit,
|
||||||
finalMapMemLimit = totalMemLimit - finalReduceMemLimit;
|
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 {
|
} else {
|
||||||
finalMapMemLimit = idealMapMemLimit;
|
finalMapResourceLimit = ideaMapResourceLimit;
|
||||||
finalReduceMemLimit = idealReduceMemLimit;
|
finalReduceResourceLimit = idealReduceResourceLimit;
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("completedMapPercent " + completedMapPercent +
|
LOG.info("completedMapPercent " + completedMapPercent
|
||||||
" totalMemLimit:" + totalMemLimit +
|
+ " totalResourceLimit:" + totalResourceLimit
|
||||||
" finalMapMemLimit:" + finalMapMemLimit +
|
+ " finalMapResourceLimit:" + finalMapResourceLimit
|
||||||
" finalReduceMemLimit:" + finalReduceMemLimit +
|
+ " finalReduceResourceLimit:" + finalReduceResourceLimit
|
||||||
" netScheduledMapMem:" + netScheduledMapMem +
|
+ " netScheduledMapResource:" + netScheduledMapResource
|
||||||
" netScheduledReduceMem:" + netScheduledReduceMem);
|
+ " netScheduledReduceResource:" + netScheduledReduceResource);
|
||||||
|
|
||||||
int rampUp =
|
int rampUp =
|
||||||
(finalReduceMemLimit - netScheduledReduceMem) / reduceResourceReqt;
|
ResourceCalculatorUtils.computeAvailableContainers(Resources.subtract(
|
||||||
|
finalReduceResourceLimit, netScheduledReduceResource),
|
||||||
|
reduceResourceReqt, getSchedulerResourceTypes());
|
||||||
|
|
||||||
if (rampUp > 0) {
|
if (rampUp > 0) {
|
||||||
rampUp = Math.min(rampUp, numPendingReduces);
|
rampUp = Math.min(rampUp, numPendingReduces);
|
||||||
LOG.info("Ramping up " + rampUp);
|
LOG.info("Ramping up " + rampUp);
|
||||||
rampUpReduces(rampUp);
|
rampUpReduces(rampUp);
|
||||||
} else if (rampUp < 0){
|
} else if (rampUp < 0) {
|
||||||
int rampDown = -1 * rampUp;
|
int rampDown = -1 * rampUp;
|
||||||
rampDown = Math.min(rampDown, scheduledReduces);
|
rampDown = Math.min(rampDown, scheduledReduces);
|
||||||
LOG.info("Ramping down " + rampDown);
|
LOG.info("Ramping down " + rampDown);
|
||||||
|
@ -618,8 +661,10 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private List<Container> getResources() throws Exception {
|
private List<Container> getResources() throws Exception {
|
||||||
int headRoom = getAvailableResources() != null
|
// will be null the first time
|
||||||
? getAvailableResources().getMemory() : 0;//first time it would be null
|
Resource headRoom =
|
||||||
|
getAvailableResources() == null ? Resources.none() :
|
||||||
|
Resources.clone(getAvailableResources());
|
||||||
AllocateResponse response;
|
AllocateResponse response;
|
||||||
/*
|
/*
|
||||||
* If contact with RM is lost, the AM will wait MR_AM_TO_RM_WAIT_INTERVAL_MS
|
* 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);
|
throw new YarnRuntimeException(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
|
Resource newHeadRoom =
|
||||||
|
getAvailableResources() == null ? Resources.none()
|
||||||
|
: getAvailableResources();
|
||||||
List<Container> newContainers = response.getAllocatedContainers();
|
List<Container> newContainers = response.getAllocatedContainers();
|
||||||
// Setting NMTokens
|
// Setting NMTokens
|
||||||
if (response.getNMTokens() != null) {
|
if (response.getNMTokens() != null) {
|
||||||
|
@ -694,10 +741,11 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
new PreemptionContext(assignedRequests), preemptReq);
|
new PreemptionContext(assignedRequests), preemptReq);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
|
if (newContainers.size() + finishedContainers.size() > 0
|
||||||
|
|| !headRoom.equals(newHeadRoom)) {
|
||||||
//something changed
|
//something changed
|
||||||
recalculateReduceSchedule = true;
|
recalculateReduceSchedule = true;
|
||||||
if (LOG.isDebugEnabled() && headRoom != newHeadRoom) {
|
if (LOG.isDebugEnabled() && !headRoom.equals(newHeadRoom)) {
|
||||||
LOG.debug("headroom=" + newHeadRoom);
|
LOG.debug("headroom=" + newHeadRoom);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -802,10 +850,18 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
}
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
public int getMemLimit() {
|
public Resource getResourceLimit() {
|
||||||
int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
|
Resource headRoom = getAvailableResources();
|
||||||
return headRoom + assignedRequests.maps.size() * mapResourceRequest +
|
if (headRoom == null) {
|
||||||
assignedRequests.reduces.size() * reduceResourceRequest;
|
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
|
@Private
|
||||||
|
@ -914,10 +970,11 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
// a container to be assigned
|
// a container to be assigned
|
||||||
boolean isAssignable = true;
|
boolean isAssignable = true;
|
||||||
Priority priority = allocated.getPriority();
|
Priority priority = allocated.getPriority();
|
||||||
int allocatedMemory = allocated.getResource().getMemory();
|
Resource allocatedResource = allocated.getResource();
|
||||||
if (PRIORITY_FAST_FAIL_MAP.equals(priority)
|
if (PRIORITY_FAST_FAIL_MAP.equals(priority)
|
||||||
|| PRIORITY_MAP.equals(priority)) {
|
|| PRIORITY_MAP.equals(priority)) {
|
||||||
if (allocatedMemory < mapResourceRequest
|
if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
|
||||||
|
mapResourceRequest, getSchedulerResourceTypes()) <= 0
|
||||||
|| maps.isEmpty()) {
|
|| maps.isEmpty()) {
|
||||||
LOG.info("Cannot assign container " + allocated
|
LOG.info("Cannot assign container " + allocated
|
||||||
+ " for a map as either "
|
+ " for a map as either "
|
||||||
|
@ -928,7 +985,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (PRIORITY_REDUCE.equals(priority)) {
|
else if (PRIORITY_REDUCE.equals(priority)) {
|
||||||
if (allocatedMemory < reduceResourceRequest
|
if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
|
||||||
|
reduceResourceRequest, getSchedulerResourceTypes()) <= 0
|
||||||
|| reduces.isEmpty()) {
|
|| reduces.isEmpty()) {
|
||||||
LOG.info("Cannot assign container " + allocated
|
LOG.info("Cannot assign container " + allocated
|
||||||
+ " for a reduce as either "
|
+ " for a reduce as either "
|
||||||
|
|
|
@ -0,0 +1,52 @@
|
||||||
|
/**
|
||||||
|
* 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.mapreduce.v2.app.rm;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
||||||
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
|
public class ResourceCalculatorUtils {
|
||||||
|
public static int divideAndCeil(int a, int b) {
|
||||||
|
if (b == 0) {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
return (a + (b - 1)) / b;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int computeAvailableContainers(Resource available,
|
||||||
|
Resource required, EnumSet<SchedulerResourceTypes> resourceTypes) {
|
||||||
|
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
||||||
|
return Math.min(available.getMemory() / required.getMemory(),
|
||||||
|
available.getVirtualCores() / required.getVirtualCores());
|
||||||
|
}
|
||||||
|
return available.getMemory() / required.getMemory();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int divideAndCeilContainers(Resource required, Resource factor,
|
||||||
|
EnumSet<SchedulerResourceTypes> resourceTypes) {
|
||||||
|
if (resourceTypes.contains(SchedulerResourceTypes.CPU)) {
|
||||||
|
return Math.max(divideAndCeil(required.getMemory(), factor.getMemory()),
|
||||||
|
divideAndCeil(required.getVirtualCores(), factor.getVirtualCores()));
|
||||||
|
}
|
||||||
|
return divideAndCeil(required.getMemory(), factor.getMemory());
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,6 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.mapreduce.v2.app.rm;
|
package org.apache.hadoop.mapreduce.v2.app.rm;
|
||||||
|
|
||||||
|
import static org.mockito.Matchers.any;
|
||||||
import static org.mockito.Matchers.anyFloat;
|
import static org.mockito.Matchers.anyFloat;
|
||||||
import static org.mockito.Matchers.anyInt;
|
import static org.mockito.Matchers.anyInt;
|
||||||
import static org.mockito.Matchers.isA;
|
import static org.mockito.Matchers.isA;
|
||||||
|
@ -30,19 +31,14 @@ import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
|
import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
|
||||||
import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
import org.apache.hadoop.mapreduce.v2.app.MRApp;
|
||||||
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
|
||||||
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -458,8 +454,8 @@ public class TestRMContainerAllocator {
|
||||||
0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
|
0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
|
||||||
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
||||||
appAttemptId, mockJob, new SystemClock());
|
appAttemptId, mockJob, new SystemClock());
|
||||||
allocator.setMapResourceRequest(1024);
|
allocator.setMapResourceRequest(BuilderUtils.newResource(1024, 1));
|
||||||
allocator.setReduceResourceRequest(1024);
|
allocator.setReduceResourceRequest(BuilderUtils.newResource(1024, 1));
|
||||||
RMContainerAllocator.AssignedRequests assignedRequests =
|
RMContainerAllocator.AssignedRequests assignedRequests =
|
||||||
allocator.getAssignedRequests();
|
allocator.getAssignedRequests();
|
||||||
RMContainerAllocator.ScheduledRequests scheduledRequests =
|
RMContainerAllocator.ScheduledRequests scheduledRequests =
|
||||||
|
@ -478,7 +474,7 @@ public class TestRMContainerAllocator {
|
||||||
|
|
||||||
@Test(timeout = 30000)
|
@Test(timeout = 30000)
|
||||||
public void testNonAggressivelyPreemptReducers() throws Exception {
|
public void testNonAggressivelyPreemptReducers() throws Exception {
|
||||||
LOG.info("Running testPreemptReducers");
|
LOG.info("Running testNonAggressivelyPreemptReducers");
|
||||||
|
|
||||||
final int preemptThreshold = 2; //sec
|
final int preemptThreshold = 2; //sec
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
|
@ -513,8 +509,8 @@ public class TestRMContainerAllocator {
|
||||||
clock.setTime(1);
|
clock.setTime(1);
|
||||||
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
|
||||||
appAttemptId, mockJob, clock);
|
appAttemptId, mockJob, clock);
|
||||||
allocator.setMapResourceRequest(1024);
|
allocator.setMapResourceRequest(BuilderUtils.newResource(1024, 1));
|
||||||
allocator.setReduceResourceRequest(1024);
|
allocator.setReduceResourceRequest(BuilderUtils.newResource(1024, 1));
|
||||||
RMContainerAllocator.AssignedRequests assignedRequests =
|
RMContainerAllocator.AssignedRequests assignedRequests =
|
||||||
allocator.getAssignedRequests();
|
allocator.getAssignedRequests();
|
||||||
RMContainerAllocator.ScheduledRequests scheduledRequests =
|
RMContainerAllocator.ScheduledRequests scheduledRequests =
|
||||||
|
@ -1774,17 +1770,19 @@ public class TestRMContainerAllocator {
|
||||||
int scheduledReduces = 0;
|
int scheduledReduces = 0;
|
||||||
int assignedMaps = 2;
|
int assignedMaps = 2;
|
||||||
int assignedReduces = 0;
|
int assignedReduces = 0;
|
||||||
int mapResourceReqt = 1024;
|
Resource mapResourceReqt = BuilderUtils.newResource(1024, 1);
|
||||||
int reduceResourceReqt = 2*1024;
|
Resource reduceResourceReqt = BuilderUtils.newResource(2 * 1024, 1);
|
||||||
int numPendingReduces = 4;
|
int numPendingReduces = 4;
|
||||||
float maxReduceRampupLimit = 0.5f;
|
float maxReduceRampupLimit = 0.5f;
|
||||||
float reduceSlowStart = 0.2f;
|
float reduceSlowStart = 0.2f;
|
||||||
|
|
||||||
RMContainerAllocator allocator = mock(RMContainerAllocator.class);
|
RMContainerAllocator allocator = mock(RMContainerAllocator.class);
|
||||||
doCallRealMethod().when(allocator).
|
doCallRealMethod().when(allocator).scheduleReduces(anyInt(), anyInt(),
|
||||||
scheduleReduces(anyInt(), anyInt(), anyInt(), anyInt(), anyInt(),
|
anyInt(), anyInt(), anyInt(), anyInt(), any(Resource.class),
|
||||||
anyInt(), anyInt(), anyInt(), anyInt(), anyFloat(), anyFloat());
|
any(Resource.class), anyInt(), anyFloat(), anyFloat());
|
||||||
|
doReturn(EnumSet.of(SchedulerResourceTypes.MEMORY)).when(allocator)
|
||||||
|
.getSchedulerResourceTypes();
|
||||||
|
|
||||||
// Test slow-start
|
// Test slow-start
|
||||||
allocator.scheduleReduces(
|
allocator.scheduleReduces(
|
||||||
totalMaps, succeededMaps,
|
totalMaps, succeededMaps,
|
||||||
|
@ -1808,6 +1806,7 @@ public class TestRMContainerAllocator {
|
||||||
verify(allocator, never()).scheduleAllReduces();
|
verify(allocator, never()).scheduleAllReduces();
|
||||||
|
|
||||||
succeededMaps = 3;
|
succeededMaps = 3;
|
||||||
|
doReturn(BuilderUtils.newResource(0, 0)).when(allocator).getResourceLimit();
|
||||||
allocator.scheduleReduces(
|
allocator.scheduleReduces(
|
||||||
totalMaps, succeededMaps,
|
totalMaps, succeededMaps,
|
||||||
scheduledMaps, scheduledReduces,
|
scheduledMaps, scheduledReduces,
|
||||||
|
@ -1818,7 +1817,8 @@ public class TestRMContainerAllocator {
|
||||||
verify(allocator, times(1)).setIsReduceStarted(true);
|
verify(allocator, times(1)).setIsReduceStarted(true);
|
||||||
|
|
||||||
// Test reduce ramp-up
|
// Test reduce ramp-up
|
||||||
doReturn(100 * 1024).when(allocator).getMemLimit();
|
doReturn(BuilderUtils.newResource(100 * 1024, 100 * 1)).when(allocator)
|
||||||
|
.getResourceLimit();
|
||||||
allocator.scheduleReduces(
|
allocator.scheduleReduces(
|
||||||
totalMaps, succeededMaps,
|
totalMaps, succeededMaps,
|
||||||
scheduledMaps, scheduledReduces,
|
scheduledMaps, scheduledReduces,
|
||||||
|
@ -1831,13 +1831,14 @@ public class TestRMContainerAllocator {
|
||||||
|
|
||||||
// Test reduce ramp-down
|
// Test reduce ramp-down
|
||||||
scheduledReduces = 3;
|
scheduledReduces = 3;
|
||||||
doReturn(10 * 1024).when(allocator).getMemLimit();
|
doReturn(BuilderUtils.newResource(10 * 1024, 10 * 1)).when(allocator)
|
||||||
|
.getResourceLimit();
|
||||||
allocator.scheduleReduces(
|
allocator.scheduleReduces(
|
||||||
totalMaps, succeededMaps,
|
totalMaps, succeededMaps,
|
||||||
scheduledMaps, scheduledReduces,
|
scheduledMaps, scheduledReduces,
|
||||||
assignedMaps, assignedReduces,
|
assignedMaps, assignedReduces,
|
||||||
mapResourceReqt, reduceResourceReqt,
|
mapResourceReqt, reduceResourceReqt,
|
||||||
numPendingReduces,
|
numPendingReduces,
|
||||||
maxReduceRampupLimit, reduceSlowStart);
|
maxReduceRampupLimit, reduceSlowStart);
|
||||||
verify(allocator).rampDownReduces(anyInt());
|
verify(allocator).rampDownReduces(anyInt());
|
||||||
|
|
||||||
|
@ -1846,7 +1847,8 @@ public class TestRMContainerAllocator {
|
||||||
// should be invoked twice.
|
// should be invoked twice.
|
||||||
scheduledMaps = 2;
|
scheduledMaps = 2;
|
||||||
assignedReduces = 2;
|
assignedReduces = 2;
|
||||||
doReturn(10 * 1024).when(allocator).getMemLimit();
|
doReturn(BuilderUtils.newResource(10 * 1024, 10 * 1)).when(allocator)
|
||||||
|
.getResourceLimit();
|
||||||
allocator.scheduleReduces(
|
allocator.scheduleReduces(
|
||||||
totalMaps, succeededMaps,
|
totalMaps, succeededMaps,
|
||||||
scheduledMaps, scheduledReduces,
|
scheduledMaps, scheduledReduces,
|
||||||
|
@ -1855,6 +1857,30 @@ public class TestRMContainerAllocator {
|
||||||
numPendingReduces,
|
numPendingReduces,
|
||||||
maxReduceRampupLimit, reduceSlowStart);
|
maxReduceRampupLimit, reduceSlowStart);
|
||||||
verify(allocator, times(2)).rampDownReduces(anyInt());
|
verify(allocator, times(2)).rampDownReduces(anyInt());
|
||||||
|
|
||||||
|
doReturn(
|
||||||
|
EnumSet.of(SchedulerResourceTypes.MEMORY, SchedulerResourceTypes.CPU))
|
||||||
|
.when(allocator).getSchedulerResourceTypes();
|
||||||
|
|
||||||
|
// Test ramp-down when enough memory but not enough cpu resource
|
||||||
|
scheduledMaps = 10;
|
||||||
|
assignedReduces = 0;
|
||||||
|
doReturn(BuilderUtils.newResource(100 * 1024, 5 * 1)).when(allocator)
|
||||||
|
.getResourceLimit();
|
||||||
|
allocator.scheduleReduces(totalMaps, succeededMaps, scheduledMaps,
|
||||||
|
scheduledReduces, assignedMaps, assignedReduces, mapResourceReqt,
|
||||||
|
reduceResourceReqt, numPendingReduces, maxReduceRampupLimit,
|
||||||
|
reduceSlowStart);
|
||||||
|
verify(allocator, times(3)).rampDownReduces(anyInt());
|
||||||
|
|
||||||
|
// Test ramp-down when enough cpu but not enough memory resource
|
||||||
|
doReturn(BuilderUtils.newResource(10 * 1024, 100 * 1)).when(allocator)
|
||||||
|
.getResourceLimit();
|
||||||
|
allocator.scheduleReduces(totalMaps, succeededMaps, scheduledMaps,
|
||||||
|
scheduledReduces, assignedMaps, assignedReduces, mapResourceReqt,
|
||||||
|
reduceResourceReqt, numPendingReduces, maxReduceRampupLimit,
|
||||||
|
reduceSlowStart);
|
||||||
|
verify(allocator, times(4)).rampDownReduces(anyInt());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class RecalculateContainerAllocator extends MyContainerAllocator {
|
private static class RecalculateContainerAllocator extends MyContainerAllocator {
|
||||||
|
@ -1868,7 +1894,7 @@ public class TestRMContainerAllocator {
|
||||||
@Override
|
@Override
|
||||||
public void scheduleReduces(int totalMaps, int completedMaps,
|
public void scheduleReduces(int totalMaps, int completedMaps,
|
||||||
int scheduledMaps, int scheduledReduces, int assignedMaps,
|
int scheduledMaps, int scheduledReduces, int assignedMaps,
|
||||||
int assignedReduces, int mapResourceReqt, int reduceResourceReqt,
|
int assignedReduces, Resource mapResourceReqt, Resource reduceResourceReqt,
|
||||||
int numPendingReduces, float maxReduceRampupLimit, float reduceSlowStart) {
|
int numPendingReduces, float maxReduceRampupLimit, float reduceSlowStart) {
|
||||||
recalculatedReduceSchedule = true;
|
recalculatedReduceSchedule = true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,12 +18,12 @@
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.util.resource;
|
package org.apache.hadoop.yarn.util.resource;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
@Private
|
@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
|
||||||
@Unstable
|
@Unstable
|
||||||
public class Resources {
|
public class Resources {
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue