|
@@ -42,6 +42,7 @@ import org.apache.hadoop.metrics.MetricsContext;
|
|
import org.apache.hadoop.metrics.MetricsRecord;
|
|
import org.apache.hadoop.metrics.MetricsRecord;
|
|
import org.apache.hadoop.metrics.MetricsUtil;
|
|
import org.apache.hadoop.metrics.MetricsUtil;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
|
+import org.apache.hadoop.net.NetworkTopology;
|
|
import org.apache.hadoop.net.Node;
|
|
import org.apache.hadoop.net.Node;
|
|
|
|
|
|
/*************************************************************
|
|
/*************************************************************
|
|
@@ -74,6 +75,10 @@ class JobInProgress {
|
|
int finishedReduceTasks = 0;
|
|
int finishedReduceTasks = 0;
|
|
int failedMapTasks = 0;
|
|
int failedMapTasks = 0;
|
|
int failedReduceTasks = 0;
|
|
int failedReduceTasks = 0;
|
|
|
|
+
|
|
|
|
+ private static float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f;
|
|
|
|
+ int completedMapsForReduceSlowstart = 0;
|
|
|
|
+
|
|
// runningMapTasks include speculative tasks, so we need to capture
|
|
// runningMapTasks include speculative tasks, so we need to capture
|
|
// speculative tasks separately
|
|
// speculative tasks separately
|
|
int speculativeMapTasks = 0;
|
|
int speculativeMapTasks = 0;
|
|
@@ -109,8 +114,22 @@ class JobInProgress {
|
|
// A set of running reduce TIPs
|
|
// A set of running reduce TIPs
|
|
Set<TaskInProgress> runningReduces;
|
|
Set<TaskInProgress> runningReduces;
|
|
|
|
|
|
- private int maxLevel;
|
|
|
|
|
|
+ private final int maxLevel;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * A special value indicating that
|
|
|
|
+ * {@link #findNewMapTask(TaskTrackerStatus, int, int, int, double)} should
|
|
|
|
+ * schedule any available map tasks for this job, including speculative tasks.
|
|
|
|
+ */
|
|
|
|
+ private final int anyCacheLevel;
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * A special value indicating that
|
|
|
|
+ * {@link #findNewMapTask(TaskTrackerStatus, int, int, int, double)} should
|
|
|
|
+ * schedule any only off-switch and speculative map tasks for this job.
|
|
|
|
+ */
|
|
|
|
+ private static final int NON_LOCAL_CACHE_LEVEL = -1;
|
|
|
|
+
|
|
private int taskCompletionEventTracker = 0;
|
|
private int taskCompletionEventTracker = 0;
|
|
List<TaskCompletionEvent> taskCompletionEvents;
|
|
List<TaskCompletionEvent> taskCompletionEvents;
|
|
|
|
|
|
@@ -185,6 +204,8 @@ class JobInProgress {
|
|
this.jobId = jobid;
|
|
this.jobId = jobid;
|
|
this.numMapTasks = conf.getNumMapTasks();
|
|
this.numMapTasks = conf.getNumMapTasks();
|
|
this.numReduceTasks = conf.getNumReduceTasks();
|
|
this.numReduceTasks = conf.getNumReduceTasks();
|
|
|
|
+ this.maxLevel = NetworkTopology.DEFAULT_HOST_LEVEL;
|
|
|
|
+ this.anyCacheLevel = this.maxLevel+1;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -240,6 +261,7 @@ class JobInProgress {
|
|
hasSpeculativeMaps = conf.getMapSpeculativeExecution();
|
|
hasSpeculativeMaps = conf.getMapSpeculativeExecution();
|
|
hasSpeculativeReduces = conf.getReduceSpeculativeExecution();
|
|
hasSpeculativeReduces = conf.getReduceSpeculativeExecution();
|
|
this.maxLevel = jobtracker.getNumTaskCacheLevels();
|
|
this.maxLevel = jobtracker.getNumTaskCacheLevels();
|
|
|
|
+ this.anyCacheLevel = this.maxLevel+1;
|
|
this.nonLocalMaps = new LinkedList<TaskInProgress>();
|
|
this.nonLocalMaps = new LinkedList<TaskInProgress>();
|
|
this.nonLocalRunningMaps = new LinkedHashSet<TaskInProgress>();
|
|
this.nonLocalRunningMaps = new LinkedHashSet<TaskInProgress>();
|
|
this.runningMapCache = new IdentityHashMap<Node, Set<TaskInProgress>>();
|
|
this.runningMapCache = new IdentityHashMap<Node, Set<TaskInProgress>>();
|
|
@@ -396,7 +418,8 @@ class JobInProgress {
|
|
}
|
|
}
|
|
LOG.info("Input size for job "+ jobId + " = " + inputLength);
|
|
LOG.info("Input size for job "+ jobId + " = " + inputLength);
|
|
if (numMapTasks > 0) {
|
|
if (numMapTasks > 0) {
|
|
- LOG.info("Split info for job:" + jobId);
|
|
|
|
|
|
+ LOG.info("Split info for job:" + jobId + " with " +
|
|
|
|
+ splits.length + " splits:");
|
|
nonRunningMapCache = createCache(splits, maxLevel);
|
|
nonRunningMapCache = createCache(splits, maxLevel);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -436,6 +459,14 @@ class JobInProgress {
|
|
nonRunningReduces.add(reduces[i]);
|
|
nonRunningReduces.add(reduces[i]);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Calculate the minimum number of maps to be complete before
|
|
|
|
+ // we should start scheduling reduces
|
|
|
|
+ completedMapsForReduceSlowstart =
|
|
|
|
+ (int)Math.ceil(
|
|
|
|
+ (conf.getFloat("mapred.reduce.slowstart.completed.maps",
|
|
|
|
+ DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART) *
|
|
|
|
+ numMapTasks));
|
|
|
|
+
|
|
// create cleanup two cleanup tips, one map and one reduce.
|
|
// create cleanup two cleanup tips, one map and one reduce.
|
|
cleanup = new TaskInProgress[2];
|
|
cleanup = new TaskInProgress[2];
|
|
// cleanup map tip. This map is doesn't use split.
|
|
// cleanup map tip. This map is doesn't use split.
|
|
@@ -896,7 +927,7 @@ class JobInProgress {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- int target = findNewMapTask(tts, clusterSize, numUniqueHosts,
|
|
|
|
|
|
+ int target = findNewMapTask(tts, clusterSize, numUniqueHosts, anyCacheLevel,
|
|
status.mapProgress());
|
|
status.mapProgress());
|
|
if (target == -1) {
|
|
if (target == -1) {
|
|
return null;
|
|
return null;
|
|
@@ -910,6 +941,52 @@ class JobInProgress {
|
|
return result;
|
|
return result;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public synchronized Task obtainNewLocalMapTask(TaskTrackerStatus tts,
|
|
|
|
+ int clusterSize,
|
|
|
|
+ int numUniqueHosts)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (!tasksInited.get()) {
|
|
|
|
+ LOG.info("Cannot create task split for " + profile.getJobID());
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int target = findNewMapTask(tts, clusterSize, numUniqueHosts, maxLevel,
|
|
|
|
+ status.mapProgress());
|
|
|
|
+ if (target == -1) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Task result = maps[target].getTaskToRun(tts.getTrackerName());
|
|
|
|
+ if (result != null) {
|
|
|
|
+ addRunningTaskToTIP(maps[target], result.getTaskID(), tts, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public synchronized Task obtainNewNonLocalMapTask(TaskTrackerStatus tts,
|
|
|
|
+ int clusterSize,
|
|
|
|
+ int numUniqueHosts)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (!tasksInited.get()) {
|
|
|
|
+ LOG.info("Cannot create task split for " + profile.getJobID());
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ int target = findNewMapTask(tts, clusterSize, numUniqueHosts,
|
|
|
|
+ NON_LOCAL_CACHE_LEVEL, status.mapProgress());
|
|
|
|
+ if (target == -1) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Task result = maps[target].getTaskToRun(tts.getTrackerName());
|
|
|
|
+ if (result != null) {
|
|
|
|
+ addRunningTaskToTIP(maps[target], result.getTaskID(), tts, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Return a CleanupTask, if appropriate, to run on the given tasktracker
|
|
* Return a CleanupTask, if appropriate, to run on the given tasktracker
|
|
*
|
|
*
|
|
@@ -1038,6 +1115,10 @@ class JobInProgress {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public synchronized boolean scheduleReduces() {
|
|
|
|
+ return finishedMapTasks >= completedMapsForReduceSlowstart;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Check whether setup task can be launched for the job.
|
|
* Check whether setup task can be launched for the job.
|
|
*
|
|
*
|
|
@@ -1066,6 +1147,12 @@ class JobInProgress {
|
|
LOG.info("Cannot create task split for " + profile.getJobID());
|
|
LOG.info("Cannot create task split for " + profile.getJobID());
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Ensure we have sufficient map outputs ready to shuffle before
|
|
|
|
+ // scheduling reduces
|
|
|
|
+ if (!scheduleReduces()) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
|
|
int target = findNewReduceTask(tts, clusterSize, numUniqueHosts,
|
|
int target = findNewReduceTask(tts, clusterSize, numUniqueHosts,
|
|
status.reduceProgress());
|
|
status.reduceProgress());
|
|
@@ -1521,12 +1608,20 @@ class JobInProgress {
|
|
* @param clusterSize The number of task trackers in the cluster
|
|
* @param clusterSize The number of task trackers in the cluster
|
|
* @param numUniqueHosts The number of hosts that run task trackers
|
|
* @param numUniqueHosts The number of hosts that run task trackers
|
|
* @param avgProgress The average progress of this kind of task in this job
|
|
* @param avgProgress The average progress of this kind of task in this job
|
|
|
|
+ * @param maxCacheLevel The maximum topology level until which to schedule
|
|
|
|
+ * maps.
|
|
|
|
+ * A value of {@link #anyCacheLevel} implies any
|
|
|
|
+ * available task (node-local, rack-local, off-switch and
|
|
|
|
+ * speculative tasks).
|
|
|
|
+ * A value of {@link #NON_LOCAL_CACHE_LEVEL} implies only
|
|
|
|
+ * off-switch/speculative tasks should be scheduled.
|
|
* @return the index in tasks of the selected task (or -1 for no task)
|
|
* @return the index in tasks of the selected task (or -1 for no task)
|
|
*/
|
|
*/
|
|
- private synchronized int findNewMapTask(TaskTrackerStatus tts,
|
|
|
|
- int clusterSize,
|
|
|
|
- int numUniqueHosts,
|
|
|
|
- double avgProgress) {
|
|
|
|
|
|
+ private synchronized int findNewMapTask(final TaskTrackerStatus tts,
|
|
|
|
+ final int clusterSize,
|
|
|
|
+ final int numUniqueHosts,
|
|
|
|
+ final int maxCacheLevel,
|
|
|
|
+ final double avgProgress) {
|
|
String taskTracker = tts.getTrackerName();
|
|
String taskTracker = tts.getTrackerName();
|
|
TaskInProgress tip = null;
|
|
TaskInProgress tip = null;
|
|
|
|
|
|
@@ -1539,14 +1634,12 @@ class JobInProgress {
|
|
return -1;
|
|
return -1;
|
|
}
|
|
}
|
|
|
|
|
|
- Node node = jobtracker.getNode(tts.getHost());
|
|
|
|
- Node nodeParentAtMaxLevel = null;
|
|
|
|
-
|
|
|
|
-
|
|
|
|
|
|
+ // Check to ensure this TaskTracker has enough resources to
|
|
|
|
+ // run tasks from this job
|
|
long outSize = resourceEstimator.getEstimatedMapOutputSize();
|
|
long outSize = resourceEstimator.getEstimatedMapOutputSize();
|
|
long availSpace = tts.getResourceStatus().getAvailableSpace();
|
|
long availSpace = tts.getResourceStatus().getAvailableSpace();
|
|
if(availSpace < outSize) {
|
|
if(availSpace < outSize) {
|
|
- LOG.warn("No room for map task. Node " + node +
|
|
|
|
|
|
+ LOG.warn("No room for map task. Node " + tts.getHost() +
|
|
" has " + availSpace +
|
|
" has " + availSpace +
|
|
" bytes free; but we expect map to take " + outSize);
|
|
" bytes free; but we expect map to take " + outSize);
|
|
|
|
|
|
@@ -1568,6 +1661,8 @@ class JobInProgress {
|
|
// We fall to linear scan of the list (III above) if we have misses in the
|
|
// We fall to linear scan of the list (III above) if we have misses in the
|
|
// above caches
|
|
// above caches
|
|
|
|
|
|
|
|
+ Node node = jobtracker.getNode(tts.getHost());
|
|
|
|
+
|
|
//
|
|
//
|
|
// I) Non-running TIP :
|
|
// I) Non-running TIP :
|
|
//
|
|
//
|
|
@@ -1575,14 +1670,20 @@ class JobInProgress {
|
|
// 1. check from local node to the root [bottom up cache lookup]
|
|
// 1. check from local node to the root [bottom up cache lookup]
|
|
// i.e if the cache is available and the host has been resolved
|
|
// i.e if the cache is available and the host has been resolved
|
|
// (node!=null)
|
|
// (node!=null)
|
|
-
|
|
|
|
if (node != null) {
|
|
if (node != null) {
|
|
Node key = node;
|
|
Node key = node;
|
|
- for (int level = 0; level < maxLevel; ++level) {
|
|
|
|
|
|
+ int level = 0;
|
|
|
|
+ // maxCacheLevel might be greater than this.maxLevel if findNewMapTask is
|
|
|
|
+ // called to schedule any task (local, rack-local, off-switch or speculative)
|
|
|
|
+ // tasks or it might be NON_LOCAL_CACHE_LEVEL (i.e. -1) if findNewMapTask is
|
|
|
|
+ // (i.e. -1) if findNewMapTask is to only schedule off-switch/speculative
|
|
|
|
+ // tasks
|
|
|
|
+ int maxLevelToSchedule = Math.min(maxCacheLevel, maxLevel);
|
|
|
|
+ for (level = 0;level < maxLevelToSchedule; ++level) {
|
|
List <TaskInProgress> cacheForLevel = nonRunningMapCache.get(key);
|
|
List <TaskInProgress> cacheForLevel = nonRunningMapCache.get(key);
|
|
if (cacheForLevel != null) {
|
|
if (cacheForLevel != null) {
|
|
tip = findTaskFromList(cacheForLevel, tts,
|
|
tip = findTaskFromList(cacheForLevel, tts,
|
|
- numUniqueHosts,level == 0);
|
|
|
|
|
|
+ numUniqueHosts,level == 0);
|
|
if (tip != null) {
|
|
if (tip != null) {
|
|
// Add to running cache
|
|
// Add to running cache
|
|
scheduleMap(tip);
|
|
scheduleMap(tip);
|
|
@@ -1597,8 +1698,11 @@ class JobInProgress {
|
|
}
|
|
}
|
|
key = key.getParent();
|
|
key = key.getParent();
|
|
}
|
|
}
|
|
- // get the node parent at max level
|
|
|
|
- nodeParentAtMaxLevel = JobTracker.getParentNode(node, maxLevel - 1);
|
|
|
|
|
|
+
|
|
|
|
+ // Check if we need to only schedule a local task (node-local/rack-local)
|
|
|
|
+ if (level == maxCacheLevel) {
|
|
|
|
+ return -1;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
//2. Search breadth-wise across parents at max level for non-running
|
|
//2. Search breadth-wise across parents at max level for non-running
|
|
@@ -1609,6 +1713,10 @@ class JobInProgress {
|
|
|
|
|
|
// collection of node at max level in the cache structure
|
|
// collection of node at max level in the cache structure
|
|
Collection<Node> nodesAtMaxLevel = jobtracker.getNodesAtMaxLevel();
|
|
Collection<Node> nodesAtMaxLevel = jobtracker.getNodesAtMaxLevel();
|
|
|
|
+
|
|
|
|
+ // get the node parent at max level
|
|
|
|
+ Node nodeParentAtMaxLevel =
|
|
|
|
+ (node == null) ? null : JobTracker.getParentNode(node, maxLevel - 1);
|
|
|
|
|
|
for (Node parent : nodesAtMaxLevel) {
|
|
for (Node parent : nodesAtMaxLevel) {
|
|
|
|
|
|
@@ -1703,6 +1811,7 @@ class JobInProgress {
|
|
return tip.getIdWithinJob();
|
|
return tip.getIdWithinJob();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
return -1;
|
|
return -1;
|
|
}
|
|
}
|
|
|
|
|