|
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.Comparator;
|
|
import java.util.Comparator;
|
|
-import java.util.EnumSet;
|
|
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
@@ -47,9 +46,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
|
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.Task;
|
|
|
|
-import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
|
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
@@ -131,6 +127,7 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
private int containersReleased = 0;
|
|
private int containersReleased = 0;
|
|
private int hostLocalAssigned = 0;
|
|
private int hostLocalAssigned = 0;
|
|
private int rackLocalAssigned = 0;
|
|
private int rackLocalAssigned = 0;
|
|
|
|
+ private int lastCompletedTasks = 0;
|
|
|
|
|
|
private boolean recalculateReduceSchedule = false;
|
|
private boolean recalculateReduceSchedule = false;
|
|
private int mapResourceReqt;//memory
|
|
private int mapResourceReqt;//memory
|
|
@@ -214,11 +211,18 @@ public class RMContainerAllocator extends RMContainerRequestor
|
|
scheduledRequests.assign(allocatedContainers);
|
|
scheduledRequests.assign(allocatedContainers);
|
|
LOG.info("After Assign: " + getStat());
|
|
LOG.info("After Assign: " + getStat());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ int completedMaps = getJob().getCompletedMaps();
|
|
|
|
+ int completedTasks = completedMaps + getJob().getCompletedReduces();
|
|
|
|
+ if (lastCompletedTasks != completedTasks) {
|
|
|
|
+ lastCompletedTasks = completedTasks;
|
|
|
|
+ recalculateReduceSchedule = true;
|
|
|
|
+ }
|
|
|
|
+
|
|
if (recalculateReduceSchedule) {
|
|
if (recalculateReduceSchedule) {
|
|
preemptReducesIfNeeded();
|
|
preemptReducesIfNeeded();
|
|
scheduleReduces(
|
|
scheduleReduces(
|
|
- getJob().getTotalMaps(), getJob().getCompletedMaps(),
|
|
|
|
|
|
+ getJob().getTotalMaps(), completedMaps,
|
|
scheduledRequests.maps.size(), scheduledRequests.reduces.size(),
|
|
scheduledRequests.maps.size(), scheduledRequests.reduces.size(),
|
|
assignedRequests.maps.size(), assignedRequests.reduces.size(),
|
|
assignedRequests.maps.size(), assignedRequests.reduces.size(),
|
|
mapResourceReqt, reduceResourceReqt,
|
|
mapResourceReqt, reduceResourceReqt,
|