|
@@ -104,6 +104,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
static final Priority PRIORITY_FAST_FAIL_MAP;
|
|
|
static final Priority PRIORITY_REDUCE;
|
|
|
static final Priority PRIORITY_MAP;
|
|
|
+ static final Priority PRIORITY_OPPORTUNISTIC_MAP;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public static final String RAMPDOWN_DIAGNOSTIC = "Reducer preempted "
|
|
@@ -119,6 +120,10 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
PRIORITY_REDUCE.setPriority(10);
|
|
|
PRIORITY_MAP = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Priority.class);
|
|
|
PRIORITY_MAP.setPriority(20);
|
|
|
+ PRIORITY_OPPORTUNISTIC_MAP =
|
|
|
+ RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
|
|
|
+ Priority.class);
|
|
|
+ PRIORITY_OPPORTUNISTIC_MAP.setPriority(19);
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -226,6 +231,9 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
// Init startTime to current time. If all goes well, it will be reset after
|
|
|
// first attempt to contact RM.
|
|
|
retrystartTime = System.currentTimeMillis();
|
|
|
+ this.scheduledRequests.setNumOpportunisticMapsPer100(
|
|
|
+ conf.getInt(MRJobConfig.MR_NUM_OPPORTUNISTIC_MAPS_PER_100,
|
|
|
+ MRJobConfig.DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PER_100));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -852,6 +860,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
setRequestLimit(PRIORITY_FAST_FAIL_MAP, mapResourceRequest,
|
|
|
failedMapRequestLimit);
|
|
|
setRequestLimit(PRIORITY_MAP, mapResourceRequest, normalMapRequestLimit);
|
|
|
+ setRequestLimit(PRIORITY_OPPORTUNISTIC_MAP, mapResourceRequest,
|
|
|
+ normalMapRequestLimit);
|
|
|
}
|
|
|
|
|
|
int numScheduledReduces = scheduledRequests.reduces.size();
|
|
@@ -979,6 +989,12 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
@VisibleForTesting
|
|
|
final Map<TaskAttemptId, ContainerRequest> maps =
|
|
|
new LinkedHashMap<TaskAttemptId, ContainerRequest>();
|
|
|
+ int mapsMod100 = 0;
|
|
|
+ int numOpportunisticMapsPer100 = 0;
|
|
|
+
|
|
|
+ void setNumOpportunisticMapsPer100(int numMaps) {
|
|
|
+ this.numOpportunisticMapsPer100 = numMaps;
|
|
|
+ }
|
|
|
|
|
|
@VisibleForTesting
|
|
|
final LinkedHashMap<TaskAttemptId, ContainerRequest> reduces =
|
|
@@ -1020,34 +1036,47 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
new ContainerRequest(event, PRIORITY_FAST_FAIL_MAP,
|
|
|
mapNodeLabelExpression);
|
|
|
LOG.info("Added "+event.getAttemptID()+" to list of failed maps");
|
|
|
+ // If its an earlier Failed attempt, do not retry as OPPORTUNISTIC
|
|
|
+ maps.put(event.getAttemptID(), request);
|
|
|
+ addContainerReq(request);
|
|
|
} else {
|
|
|
- for (String host : event.getHosts()) {
|
|
|
- LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
|
|
|
- if (list == null) {
|
|
|
- list = new LinkedList<TaskAttemptId>();
|
|
|
- mapsHostMapping.put(host, list);
|
|
|
+ if (mapsMod100 < numOpportunisticMapsPer100) {
|
|
|
+ request =
|
|
|
+ new ContainerRequest(event, PRIORITY_OPPORTUNISTIC_MAP,
|
|
|
+ mapNodeLabelExpression);
|
|
|
+ maps.put(event.getAttemptID(), request);
|
|
|
+ addOpportunisticResourceRequest(request.priority, request.capability);
|
|
|
+ } else {
|
|
|
+ request =
|
|
|
+ new ContainerRequest(event, PRIORITY_MAP, mapNodeLabelExpression);
|
|
|
+ for (String host : event.getHosts()) {
|
|
|
+ LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
|
|
|
+ if (list == null) {
|
|
|
+ list = new LinkedList<TaskAttemptId>();
|
|
|
+ mapsHostMapping.put(host, list);
|
|
|
+ }
|
|
|
+ list.add(event.getAttemptID());
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Added attempt req to host " + host);
|
|
|
+ }
|
|
|
}
|
|
|
- list.add(event.getAttemptID());
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Added attempt req to host " + host);
|
|
|
+ for (String rack : event.getRacks()) {
|
|
|
+ LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
|
|
|
+ if (list == null) {
|
|
|
+ list = new LinkedList<TaskAttemptId>();
|
|
|
+ mapsRackMapping.put(rack, list);
|
|
|
+ }
|
|
|
+ list.add(event.getAttemptID());
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Added attempt req to rack " + rack);
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
- for (String rack: event.getRacks()) {
|
|
|
- LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
|
|
|
- if (list == null) {
|
|
|
- list = new LinkedList<TaskAttemptId>();
|
|
|
- mapsRackMapping.put(rack, list);
|
|
|
- }
|
|
|
- list.add(event.getAttemptID());
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Added attempt req to rack " + rack);
|
|
|
- }
|
|
|
- }
|
|
|
- request =
|
|
|
- new ContainerRequest(event, PRIORITY_MAP, mapNodeLabelExpression);
|
|
|
+ maps.put(event.getAttemptID(), request);
|
|
|
+ addContainerReq(request);
|
|
|
+ }
|
|
|
+ mapsMod100++;
|
|
|
+ mapsMod100 %= 100;
|
|
|
}
|
|
|
- maps.put(event.getAttemptID(), request);
|
|
|
- addContainerReq(request);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -1077,7 +1106,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
Priority priority = allocated.getPriority();
|
|
|
Resource allocatedResource = allocated.getResource();
|
|
|
if (PRIORITY_FAST_FAIL_MAP.equals(priority)
|
|
|
- || PRIORITY_MAP.equals(priority)) {
|
|
|
+ || PRIORITY_MAP.equals(priority)
|
|
|
+ || PRIORITY_OPPORTUNISTIC_MAP.equals(priority)) {
|
|
|
if (ResourceCalculatorUtils.computeAvailableContainers(allocatedResource,
|
|
|
mapResourceRequest, getSchedulerResourceTypes()) <= 0
|
|
|
|| maps.isEmpty()) {
|
|
@@ -1235,7 +1265,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
LOG.info("Found replacement: " + toBeReplaced);
|
|
|
return toBeReplaced;
|
|
|
}
|
|
|
- else if (PRIORITY_MAP.equals(priority)) {
|
|
|
+ else if (PRIORITY_MAP.equals(priority)
|
|
|
+ || PRIORITY_OPPORTUNISTIC_MAP.equals(priority)) {
|
|
|
LOG.info("Replacing MAP container " + allocated.getId());
|
|
|
// allocated container was for a map
|
|
|
String host = allocated.getNodeId().getHost();
|
|
@@ -1298,29 +1329,33 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
while(it.hasNext() && maps.size() > 0 && canAssignMaps()){
|
|
|
Container allocated = it.next();
|
|
|
Priority priority = allocated.getPriority();
|
|
|
- assert PRIORITY_MAP.equals(priority);
|
|
|
- // "if (maps.containsKey(tId))" below should be almost always true.
|
|
|
- // hence this while loop would almost always have O(1) complexity
|
|
|
- String host = allocated.getNodeId().getHost();
|
|
|
- LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
|
|
|
- while (list != null && list.size() > 0) {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Host matched to the request list " + host);
|
|
|
- }
|
|
|
- TaskAttemptId tId = list.removeFirst();
|
|
|
- if (maps.containsKey(tId)) {
|
|
|
- ContainerRequest assigned = maps.remove(tId);
|
|
|
- containerAssigned(allocated, assigned);
|
|
|
- it.remove();
|
|
|
- JobCounterUpdateEvent jce =
|
|
|
- new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
|
|
|
- jce.addCounterUpdate(JobCounter.DATA_LOCAL_MAPS, 1);
|
|
|
- eventHandler.handle(jce);
|
|
|
- hostLocalAssigned++;
|
|
|
+ assert (PRIORITY_MAP.equals(priority)
|
|
|
+ || PRIORITY_OPPORTUNISTIC_MAP.equals(priority));
|
|
|
+ if (!PRIORITY_OPPORTUNISTIC_MAP.equals(priority)) {
|
|
|
+ // "if (maps.containsKey(tId))" below should be almost always true.
|
|
|
+ // hence this while loop would almost always have O(1) complexity
|
|
|
+ String host = allocated.getNodeId().getHost();
|
|
|
+ LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
|
|
|
+ while (list != null && list.size() > 0) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Assigned based on host match " + host);
|
|
|
+ LOG.debug("Host matched to the request list " + host);
|
|
|
+ }
|
|
|
+ TaskAttemptId tId = list.removeFirst();
|
|
|
+ if (maps.containsKey(tId)) {
|
|
|
+ ContainerRequest assigned = maps.remove(tId);
|
|
|
+ containerAssigned(allocated, assigned);
|
|
|
+ it.remove();
|
|
|
+ JobCounterUpdateEvent jce =
|
|
|
+ new JobCounterUpdateEvent(assigned.attemptID.getTaskId()
|
|
|
+ .getJobId());
|
|
|
+ jce.addCounterUpdate(JobCounter.DATA_LOCAL_MAPS, 1);
|
|
|
+ eventHandler.handle(jce);
|
|
|
+ hostLocalAssigned++;
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Assigned based on host match " + host);
|
|
|
+ }
|
|
|
+ break;
|
|
|
}
|
|
|
- break;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1330,27 +1365,31 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
while(it.hasNext() && maps.size() > 0 && canAssignMaps()){
|
|
|
Container allocated = it.next();
|
|
|
Priority priority = allocated.getPriority();
|
|
|
- assert PRIORITY_MAP.equals(priority);
|
|
|
- // "if (maps.containsKey(tId))" below should be almost always true.
|
|
|
- // hence this while loop would almost always have O(1) complexity
|
|
|
- String host = allocated.getNodeId().getHost();
|
|
|
- String rack = RackResolver.resolve(host).getNetworkLocation();
|
|
|
- LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
|
|
|
- while (list != null && list.size() > 0) {
|
|
|
- TaskAttemptId tId = list.removeFirst();
|
|
|
- if (maps.containsKey(tId)) {
|
|
|
- ContainerRequest assigned = maps.remove(tId);
|
|
|
- containerAssigned(allocated, assigned);
|
|
|
- it.remove();
|
|
|
- JobCounterUpdateEvent jce =
|
|
|
- new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
|
|
|
- jce.addCounterUpdate(JobCounter.RACK_LOCAL_MAPS, 1);
|
|
|
- eventHandler.handle(jce);
|
|
|
- rackLocalAssigned++;
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Assigned based on rack match " + rack);
|
|
|
+ assert (PRIORITY_MAP.equals(priority)
|
|
|
+ || PRIORITY_OPPORTUNISTIC_MAP.equals(priority));
|
|
|
+ if (!PRIORITY_OPPORTUNISTIC_MAP.equals(priority)) {
|
|
|
+ // "if (maps.containsKey(tId))" below should be almost always true.
|
|
|
+ // hence this while loop would almost always have O(1) complexity
|
|
|
+ String host = allocated.getNodeId().getHost();
|
|
|
+ String rack = RackResolver.resolve(host).getNetworkLocation();
|
|
|
+ LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
|
|
|
+ while (list != null && list.size() > 0) {
|
|
|
+ TaskAttemptId tId = list.removeFirst();
|
|
|
+ if (maps.containsKey(tId)) {
|
|
|
+ ContainerRequest assigned = maps.remove(tId);
|
|
|
+ containerAssigned(allocated, assigned);
|
|
|
+ it.remove();
|
|
|
+ JobCounterUpdateEvent jce =
|
|
|
+ new JobCounterUpdateEvent(assigned.attemptID.getTaskId()
|
|
|
+ .getJobId());
|
|
|
+ jce.addCounterUpdate(JobCounter.RACK_LOCAL_MAPS, 1);
|
|
|
+ eventHandler.handle(jce);
|
|
|
+ rackLocalAssigned++;
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Assigned based on rack match " + rack);
|
|
|
+ }
|
|
|
+ break;
|
|
|
}
|
|
|
- break;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1360,7 +1399,8 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
|
while(it.hasNext() && maps.size() > 0 && canAssignMaps()){
|
|
|
Container allocated = it.next();
|
|
|
Priority priority = allocated.getPriority();
|
|
|
- assert PRIORITY_MAP.equals(priority);
|
|
|
+ assert (PRIORITY_MAP.equals(priority)
|
|
|
+ || PRIORITY_OPPORTUNISTIC_MAP.equals(priority));
|
|
|
TaskAttemptId tId = maps.keySet().iterator().next();
|
|
|
ContainerRequest assigned = maps.remove(tId);
|
|
|
containerAssigned(allocated, assigned);
|