/** * JobTracker.submitJob() kicks off a new job. * * <p>Create a 'JobInProgress' object, which contains both JobProfile and JobStatus. Those two * sub-objects are sometimes shipped outside of the JobTracker. But JobInProgress adds info that's * useful for the JobTracker alone. * * <p>We add the JIP to the jobInitQueue, which is processed asynchronously to handle * split-computation and build up the right TaskTracker/Block mapping. */ public synchronized JobStatus submitJob(String jobFile) throws IOException { totalSubmissions++; JobInProgress job = new JobInProgress(jobFile, this, this.conf); synchronized (jobs) { synchronized (jobsByArrival) { synchronized (jobInitQueue) { jobs.put(job.getProfile().getJobId(), job); jobsByArrival.add(job); jobInitQueue.add(job); jobInitQueue.notifyAll(); } } } return job.getStatus(); }
/** * We lost the task tracker! All task-tracker structures have already been updated. Just process * the contained tasks and any jobs that might be affected. */ void lostTaskTracker(String trackerName) { LOG.info("Lost tracker '" + trackerName + "'"); TreeSet lostTasks = (TreeSet) trackerToTaskMap.get(trackerName); trackerToTaskMap.remove(trackerName); if (lostTasks != null) { for (Iterator it = lostTasks.iterator(); it.hasNext(); ) { String taskId = (String) it.next(); TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(taskId); // Tell the job to fail the relevant task JobInProgress job = tip.getJob(); job.failedTask(tip, taskId, trackerName); } } }
/** * Recompute the internal variables used by the scheduler - per-job weights, fair shares, * deficits, minimum slot allocations, and numbers of running and needed tasks of each type. */ protected synchronized void update() { // Remove non-running jobs List<JobInProgress> toRemove = new ArrayList<JobInProgress>(); for (JobInProgress job : infos.keySet()) { int runState = job.getStatus().getRunState(); if (runState == JobStatus.SUCCEEDED || runState == JobStatus.FAILED || runState == JobStatus.KILLED) { toRemove.add(job); } } for (JobInProgress job : toRemove) { infos.remove(job); poolMgr.removeJob(job); } // Update running jobs with deficits since last update, and compute new // slot allocations, weight, shares and task counts long now = clock.getTime(); long timeDelta = now - lastUpdateTime; updateDeficits(timeDelta); updateRunnability(); updateTaskCounts(); updateWeights(); updateMinSlots(); updateFairShares(); lastUpdateTime = now; }
private void updateRunnability() { // Start by marking everything as not runnable for (JobInfo info : infos.values()) { info.runnable = false; } // Create a list of sorted jobs in order of start time and priority List<JobInProgress> jobs = new ArrayList<JobInProgress>(infos.keySet()); Collections.sort(jobs, new FifoJobComparator()); // Mark jobs as runnable in order of start time and priority, until // user or pool limits have been reached. Map<String, Integer> userJobs = new HashMap<String, Integer>(); Map<String, Integer> poolJobs = new HashMap<String, Integer>(); for (JobInProgress job : jobs) { if (job.getStatus().getRunState() == JobStatus.RUNNING) { String user = job.getJobConf().getUser(); String pool = poolMgr.getPoolName(job); int userCount = userJobs.containsKey(user) ? userJobs.get(user) : 0; int poolCount = poolJobs.containsKey(pool) ? poolJobs.get(pool) : 0; if (userCount < poolMgr.getUserMaxJobs(user) && poolCount < poolMgr.getPoolMaxJobs(pool)) { infos.get(job).runnable = true; userJobs.put(user, userCount + 1); poolJobs.put(pool, poolCount + 1); } } } }
/** * The run method lives for the life of the JobTracker, and removes Jobs that are not still * running, but which finished a long time ago. */ public void run() { while (shouldRun) { try { Thread.sleep(RETIRE_JOB_CHECK_INTERVAL); } catch (InterruptedException ie) { } synchronized (jobs) { synchronized (jobInitQueue) { synchronized (jobsByArrival) { for (Iterator it = jobs.keySet().iterator(); it.hasNext(); ) { String jobid = (String) it.next(); JobInProgress job = (JobInProgress) jobs.get(jobid); if (job.getStatus().getRunState() != JobStatus.RUNNING && job.getStatus().getRunState() != JobStatus.PREP && (job.getFinishTime() + RETIRE_JOB_INTERVAL < System.currentTimeMillis())) { it.remove(); jobInitQueue.remove(job); jobsByArrival.remove(job); } } } } } } }
public synchronized JobStatus getJobStatus(String jobid) { JobInProgress job = (JobInProgress) jobs.get(jobid); if (job != null) { return job.getStatus(); } else { return null; } }
public synchronized JobProfile getJobProfile(String jobid) { JobInProgress job = (JobInProgress) jobs.get(jobid); if (job != null) { return job.getProfile(); } else { return null; } }
private void printFailures( JspWriter out, JobTracker tracker, JobID jobId, String kind, String cause) throws IOException { JobInProgress job = (JobInProgress) tracker.getJob(jobId); if (job == null) { out.print("<b>Job " + jobId + " not found.</b><br>\n"); return; } boolean includeMap = false; boolean includeReduce = false; if (kind == null) { includeMap = true; includeReduce = true; } else if ("map".equals(kind)) { includeMap = true; } else if ("reduce".equals(kind)) { includeReduce = true; } else if ("all".equals(kind)) { includeMap = true; includeReduce = true; } else { out.print("<b>Kind " + kind + " not supported.</b><br>\n"); return; } TaskStatus.State state = null; try { if (cause != null) { state = TaskStatus.State.valueOf(cause.toUpperCase()); if (state != TaskStatus.State.FAILED && state != TaskStatus.State.KILLED) { out.print("<b>Cause '" + cause + "' is not an 'unsuccessful' state.</b><br>\n"); return; } } } catch (IllegalArgumentException e) { out.print("<b>Cause '" + cause + "' not supported.</b><br>\n"); return; } out.print("<table border=2 cellpadding=\"5\" cellspacing=\"2\">"); out.print( "<tr><th>Attempt</th><th>Task</th><th>Machine</th><th>State</th>" + "<th>Error</th><th>Logs</th></tr>\n"); if (includeMap) { TaskInProgress[] tips = job.getTasks(TaskType.MAP); for (int i = 0; i < tips.length; ++i) { printFailedAttempts(out, tracker, jobId, tips[i], state); } } if (includeReduce) { TaskInProgress[] tips = job.getTasks(TaskType.REDUCE); for (int i = 0; i < tips.length; ++i) { printFailedAttempts(out, tracker, jobId, tips[i], state); } } out.print("</table>\n"); }
public Vector completedJobs() { Vector v = new Vector(); for (Iterator it = jobs.values().iterator(); it.hasNext(); ) { JobInProgress jip = (JobInProgress) it.next(); JobStatus status = jip.getStatus(); if (status.getRunState() == JobStatus.SUCCEEDED) { v.add(jip); } } return v; }
public Vector runningJobs() { Vector v = new Vector(); for (Iterator it = jobs.values().iterator(); it.hasNext(); ) { JobInProgress jip = (JobInProgress) it.next(); JobStatus status = jip.getStatus(); if (status.getRunState() == JobStatus.RUNNING) { v.add(jip); } } return v; }
/** * Check the ACLs for a user doing the passed operation. * * <ul> * <li>If ACLs are disabled, allow all users. * <li>Otherwise, if the operation is not a job operation(for eg. submit-job-to-queue), then * allow only (a) clusterOwner(who started the cluster), (b) cluster administrators and (c) * members of queue-submit-job-acl for the queue. * <li>If the operation is a job operation, then allow only (a) jobOwner, (b) clusterOwner(who * started the cluster), (c) cluster administrators, (d) members of queue admins acl for the * queue and (e) members of job acl for the job operation * </ul> * * @param job the job on which operation is requested * @param callerUGI the user who is requesting the operation * @param operation the operation for which authorization is needed * @throws AccessControlException */ void checkAccess(JobInProgress job, UserGroupInformation callerUGI, Operation operation) throws AccessControlException { String queue = job.getProfile().getQueueName(); String jobId = job.getJobID().toString(); JobStatus jobStatus = job.getStatus(); String jobOwner = jobStatus.getUsername(); AccessControlList jobAcl = jobStatus.getJobACLs().get(operation.jobACLNeeded); checkAccess(jobId, callerUGI, queue, operation, jobOwner, jobAcl); }
private void updateTaskCounts() { for (Map.Entry<JobInProgress, JobInfo> entry : infos.entrySet()) { JobInProgress job = entry.getKey(); JobInfo info = entry.getValue(); if (job.getStatus().getRunState() != JobStatus.RUNNING) continue; // Job is still in PREP state and tasks aren't initialized // Count maps int totalMaps = job.numMapTasks; int finishedMaps = 0; int runningMaps = 0; for (TaskInProgress tip : job.getMapTasks()) { if (tip.isComplete()) { finishedMaps += 1; } else if (tip.isRunning()) { runningMaps += tip.getActiveTasks().size(); } } info.runningMaps = runningMaps; info.neededMaps = (totalMaps - runningMaps - finishedMaps + taskSelector.neededSpeculativeMaps(job)); // Count reduces int totalReduces = job.numReduceTasks; int finishedReduces = 0; int runningReduces = 0; for (TaskInProgress tip : job.getReduceTasks()) { if (tip.isComplete()) { finishedReduces += 1; } else if (tip.isRunning()) { runningReduces += tip.getActiveTasks().size(); } } info.runningReduces = runningReduces; info.neededReduces = (totalReduces - runningReduces - finishedReduces + taskSelector.neededSpeculativeReduces(job)); // If the job was marked as not runnable due to its user or pool having // too many active jobs, set the neededMaps/neededReduces to 0. We still // count runningMaps/runningReduces however so we can give it a deficit. if (!info.runnable) { info.neededMaps = 0; info.neededReduces = 0; } } }
public synchronized TaskReport[] getReduceTaskReports(String jobid) { JobInProgress job = (JobInProgress) jobs.get(jobid); if (job == null) { return new TaskReport[0]; } else { Vector reports = new Vector(); Vector completeReduceTasks = job.reportTasksInProgress(false, true); for (Iterator it = completeReduceTasks.iterator(); it.hasNext(); ) { TaskInProgress tip = (TaskInProgress) it.next(); reports.add(tip.generateSingleReport()); } Vector incompleteReduceTasks = job.reportTasksInProgress(false, false); for (Iterator it = incompleteReduceTasks.iterator(); it.hasNext(); ) { TaskInProgress tip = (TaskInProgress) it.next(); reports.add(tip.generateSingleReport()); } return (TaskReport[]) reports.toArray(new TaskReport[reports.size()]); } }
/** * Accept and process a new TaskTracker profile. We might have known about the TaskTracker * previously, or it might be brand-new. All task-tracker structures have already been updated. * Just process the contained tasks and any jobs that might be affected. */ void updateTaskStatuses(TaskTrackerStatus status) { for (Iterator it = status.taskReports(); it.hasNext(); ) { TaskStatus report = (TaskStatus) it.next(); TaskInProgress tip = (TaskInProgress) taskidToTIPMap.get(report.getTaskId()); if (tip == null) { LOG.info( "Serious problem. While updating status, cannot find taskid " + report.getTaskId()); } else { JobInProgress job = tip.getJob(); job.updateTaskStatus(tip, report); if (report.getRunState() == TaskStatus.SUCCEEDED) { job.completedTask(tip, report.getTaskId()); } else if (report.getRunState() == TaskStatus.FAILED) { // Tell the job to fail the relevant task job.failedTask(tip, report.getTaskId(), status.getTrackerName()); } } } }
private boolean isWait(JobInProgress job) { long bookingTime = job.getJobConf().getLong(BOOKING_TIME, 0); String[] dependencyJobs = job.getJobConf().getStrings(BOOKING_DEPENDENCY_JOBID, null); boolean bookingTimeFilter = false; boolean dependencyJobFilter = false; if (bookingTime >= System.currentTimeMillis()) { bookingTimeFilter = true; } if (null != dependencyJobs) { for (String dependencyJob : dependencyJobs) { JobStatus dependencyJobStatus = (JobStatus) finishJobStatus.get(dependencyJob); if (null != dependencyJobStatus && dependencyJobStatus.getRunState() != JobStatus.SUCCEEDED) { dependencyJobFilter = true; } } } if (bookingTimeFilter || dependencyJobFilter) return true; else return false; }
@Override public Task assignTask(TaskTrackerStatus tts, long currentTime, Collection<JobInProgress> visited) throws IOException { if (isRunnable()) { visited.add(job); TaskTrackerManager ttm = scheduler.taskTrackerManager; ClusterStatus clusterStatus = ttm.getClusterStatus(); int numTaskTrackers = clusterStatus.getTaskTrackers(); if (taskType == TaskType.MAP) { LocalityLevel localityLevel = scheduler.getAllowedLocalityLevel(job, currentTime); scheduler.getEventLog().log("ALLOWED_LOC_LEVEL", job.getJobID(), localityLevel); // obtainNewMapTask needs to be passed 1 + the desired locality level return job.obtainNewMapTask( tts, numTaskTrackers, ttm.getNumberOfUniqueHosts(), localityLevel.toCacheLevelCap()); } else { return job.obtainNewReduceTask(tts, numTaskTrackers, ttm.getNumberOfUniqueHosts()); } } else { return null; } }
@Override public void updateDemand() { demand = 0; if (isRunnable()) { // For reduces, make sure enough maps are done that reduces can launch if (taskType == TaskType.REDUCE && !job.scheduleReduces()) return; // Add up demand from each TaskInProgress; each TIP can either // - have no attempts running, in which case it demands 1 slot // - have N attempts running, in which case it demands N slots, and may // potentially demand one more slot if it needs to be speculated TaskInProgress[] tips = (taskType == TaskType.MAP ? job.getTasks(TaskType.MAP) : job.getTasks(TaskType.REDUCE)); boolean speculationEnabled = (taskType == TaskType.MAP ? job.hasSpeculativeMaps() : job.hasSpeculativeReduces()); long time = scheduler.getClock().getTime(); for (TaskInProgress tip : tips) { if (!tip.isComplete()) { if (tip.isRunning()) { // Count active tasks and any speculative task we want to launch demand += tip.getActiveTasks().size(); if (speculationEnabled && tip.hasSpeculativeTask(time, job.getStatus().mapProgress())) demand += 1; } else { // Need to launch 1 task demand += 1; } } } } }
public int compare(JobInProgress j1, JobInProgress j2) { // Put needy jobs ahead of non-needy jobs (where needy means must receive // new tasks to meet slot minimum), comparing among jobs of the same type // by deficit so as to put jobs with higher deficit ahead. JobInfo j1Info = infos.get(j1); JobInfo j2Info = infos.get(j2); long deficitDif; boolean j1Needy, j2Needy; if (taskType == TaskType.MAP) { j1Needy = j1.runningMaps() < Math.floor(j1Info.minMaps); j2Needy = j2.runningMaps() < Math.floor(j2Info.minMaps); deficitDif = j2Info.mapDeficit - j1Info.mapDeficit; } else { j1Needy = j1.runningReduces() < Math.floor(j1Info.minReduces); j2Needy = j2.runningReduces() < Math.floor(j2Info.minReduces); deficitDif = j2Info.reduceDeficit - j1Info.reduceDeficit; } if (j1Needy && !j2Needy) return -1; else if (j2Needy && !j1Needy) return 1; else // Both needy or both non-needy; compare by deficit return (int) Math.signum(deficitDif); }
public void run() { while (shouldRun) { JobInProgress job = null; synchronized (jobInitQueue) { if (jobInitQueue.size() > 0) { job = (JobInProgress) jobInitQueue.elementAt(0); jobInitQueue.remove(job); } else { try { jobInitQueue.wait(JOBINIT_SLEEP_INTERVAL); } catch (InterruptedException iex) { } } } try { if (job != null) { job.initTasks(); } } catch (Exception e) { LOG.log(Level.WARNING, "job init failed", e); job.kill(); } } }
private boolean exceededPadding( boolean isMapTask, ClusterStatus clusterStatus, int maxTaskTrackerSlots) { int numTaskTrackers = clusterStatus.getTaskTrackers(); int totalTasks = (isMapTask) ? clusterStatus.getMapTasks() : clusterStatus.getReduceTasks(); int totalTaskCapacity = isMapTask ? clusterStatus.getMaxMapTasks() : clusterStatus.getMaxReduceTasks(); Collection<JobInProgress> jobQueue = jobQueueJobInProgressListener.getJobQueue(); boolean exceededPadding = false; synchronized (jobQueue) { int totalNeededTasks = 0; for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() != JobStatus.RUNNING || job.numReduceTasks == 0) { continue; } // // Beyond the highest-priority task, reserve a little // room for failures and speculative executions; don't // schedule tasks to the hilt. // totalNeededTasks += isMapTask ? job.desiredMaps() : job.desiredReduces(); int padding = 0; if (numTaskTrackers > MIN_CLUSTER_SIZE_FOR_PADDING) { padding = Math.min(maxTaskTrackerSlots, (int) (totalNeededTasks * padFraction)); } if (totalTasks + padding >= totalTaskCapacity) { exceededPadding = true; break; } } } return exceededPadding; }
private double calculateWeight(JobInProgress job, TaskType taskType) { if (!isRunnable(job)) { return 0; } else { double weight = 1.0; if (sizeBasedWeight) { // Set weight based on runnable tasks weight = Math.log1p(runnableTasks(job, taskType)) / Math.log(2); } weight *= getPriorityFactor(job.getPriority()); if (weightAdjuster != null) { // Run weight through the user-supplied weightAdjuster weight = weightAdjuster.adjustWeight(job, taskType, weight); } return weight; } }
// create a new TaskInProgress and make it running by adding it to jobtracker private TaskInProgress createAndAddTIP(JobTracker jobtracker, JobInProgress jip, TaskType type) { JobConf conf = jip.getJobConf(); JobID id = jip.getJobID(); // now create a fake tip for this fake job TaskInProgress tip = null; if (type == TaskType.MAP) { tip = new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT, jobtracker, conf, jip, 0, 1); jip.maps = new TaskInProgress[] {tip}; } else if (type == TaskType.REDUCE) { tip = new TaskInProgress(id, "dummy", jip.desiredMaps(), 0, jobtracker, conf, jip, 1); jip.reduces = new TaskInProgress[] {tip}; } else if (type == TaskType.JOB_SETUP) { tip = new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT, jobtracker, conf, jip, 0, 1); jip.setup = new TaskInProgress[] {tip}; } else if (type == TaskType.JOB_CLEANUP) { tip = new TaskInProgress(id, "dummy", JobSplit.EMPTY_TASK_SPLIT, jobtracker, conf, jip, 0, 1); jip.cleanup = new TaskInProgress[] {tip}; } return tip; }
@Override protected void updateRunnability() { // Start by marking everything as not runnable for (JobInfo info : infos.values()) { info.runnable = false; } // Create a list of sorted jobs in order of start time and priority List<JobInProgress> jobs = new ArrayList<JobInProgress>(infos.keySet()); Collections.sort(jobs, new FifoJobComparator()); // Mark jobs as runnable in order of start time and priority, until // user or pool limits have been reached. Map<String, Integer> userJobs = new HashMap<String, Integer>(); Map<String, Integer> poolJobs = new HashMap<String, Integer>(); for (JobInProgress job : jobs) { if (isWait(job)) { LOG.debug("Booked job. It's waiting... : " + job.getJobID()); continue; } String user = job.getJobConf().getUser(); String pool = poolMgr.getPoolName(job); int userCount = userJobs.containsKey(user) ? userJobs.get(user) : 0; int poolCount = poolJobs.containsKey(pool) ? poolJobs.get(pool) : 0; if (userCount < poolMgr.getUserMaxJobs(user) && poolCount < poolMgr.getPoolMaxJobs(pool)) { if (job.getStatus().getRunState() == JobStatus.RUNNING || job.getStatus().getRunState() == JobStatus.PREP) { userJobs.put(user, userCount + 1); poolJobs.put(pool, poolCount + 1); JobInfo jobInfo = infos.get(job); if (job.getStatus().getRunState() == JobStatus.RUNNING) { jobInfo.runnable = true; } else { // The job is in the PREP state. Give it to the job initializer // for initialization if we have not already done it. if (jobInfo.needsInitializing) { jobInfo.needsInitializing = false; jobInitializer.initJob(jobInfo, job); } } } } } }
public void _jspService(HttpServletRequest request, HttpServletResponse response) throws java.io.IOException, ServletException { PageContext pageContext = null; HttpSession session = null; ServletContext application = null; ServletConfig config = null; JspWriter out = null; Object page = this; JspWriter _jspx_out = null; PageContext _jspx_page_context = null; try { response.setContentType("text/html; charset=UTF-8"); pageContext = _jspxFactory.getPageContext(this, request, response, null, true, 8192, true); _jspx_page_context = pageContext; application = pageContext.getServletContext(); config = pageContext.getServletConfig(); session = pageContext.getSession(); out = pageContext.getOut(); _jspx_out = out; _jspx_resourceInjector = (org.apache.jasper.runtime.ResourceInjector) application.getAttribute("com.sun.appserv.jsp.resource.injector"); out.write('\n'); JobTracker tracker = (JobTracker) application.getAttribute("job.tracker"); ClusterStatus status = tracker.getClusterStatus(); String trackerName = StringUtils.simpleHostname(tracker.getJobTrackerMachine()); out.write("\n<html>\n<head>\n<title>"); out.print(trackerName); out.write( " Hadoop Locality Statistics</title>\n<link rel=\"stylesheet\" type=\"text/css\" href=\"/static/hadoop.css\">\n</head>\n<body>\n<h1>"); out.print(trackerName); out.write(" Hadoop Locality Statistics</h1>\n\n<b>State:</b> "); out.print(status.getJobTrackerState()); out.write("<br>\n<b>Started:</b> "); out.print(new Date(tracker.getStartTime())); out.write("<br>\n<b>Version:</b> "); out.print(VersionInfo.getVersion()); out.write(",\n r"); out.print(VersionInfo.getRevision()); out.write("<br>\n<b>Compiled:</b> "); out.print(VersionInfo.getDate()); out.write(" by\n "); out.print(VersionInfo.getUser()); out.write("<br>\n<b>Identifier:</b> "); out.print(tracker.getTrackerIdentifier()); out.write("<br>\n\n<hr>\n\n"); Collection<JobInProgress> jobs = new ArrayList<JobInProgress>(); jobs.addAll(tracker.completedJobs()); jobs.addAll(tracker.runningJobs()); jobs.addAll(tracker.failedJobs()); int dataLocalMaps = 0; int rackLocalMaps = 0; int totalMaps = 0; int totalReduces = 0; for (JobInProgress job : jobs) { Counters counters = job.getCounters(); dataLocalMaps += counters.getCounter(JobInProgress.Counter.DATA_LOCAL_MAPS); rackLocalMaps += counters.getCounter(JobInProgress.Counter.RACK_LOCAL_MAPS); totalMaps += counters.getCounter(JobInProgress.Counter.TOTAL_LAUNCHED_MAPS); totalReduces += counters.getCounter(JobInProgress.Counter.TOTAL_LAUNCHED_REDUCES); } int dataLocalMapPct = totalMaps == 0 ? 0 : (100 * dataLocalMaps) / totalMaps; int rackLocalMapPct = totalMaps == 0 ? 0 : (100 * rackLocalMaps) / totalMaps; int dataRackLocalMapPct = totalMaps == 0 ? 0 : (100 * (dataLocalMaps + rackLocalMaps)) / totalMaps; out.write("\n<p>\n<b>Data Local Maps:</b> "); out.print(dataLocalMaps); out.write(' '); out.write('('); out.print(dataLocalMapPct); out.write("%) <br>\n<b>Rack Local Maps:</b> "); out.print(rackLocalMaps); out.write(' '); out.write('('); out.print(rackLocalMapPct); out.write("%) <br>\n<b>Data or Rack Local:</b> "); out.print(dataLocalMaps + rackLocalMaps); out.write(' '); out.write('('); out.print(dataRackLocalMapPct); out.write("%) <br>\n<b>Total Maps:</b> "); out.print(totalMaps); out.write(" <br>\n<b>Total Reduces:</b> "); out.print(totalReduces); out.write(" <br>\n</p>\n\n"); out.println(ServletUtil.htmlFooter()); out.write('\n'); } catch (Throwable t) { if (!(t instanceof SkipPageException)) { out = _jspx_out; if (out != null && out.getBufferSize() != 0) out.clearBuffer(); if (_jspx_page_context != null) _jspx_page_context.handlePageException(t); } } finally { _jspxFactory.releasePageContext(_jspx_page_context); } }
@Override public synchronized List<Task> assignTasks(TaskTracker taskTracker) throws IOException { TaskTrackerStatus taskTrackerStatus = taskTracker.getStatus(); ClusterStatus clusterStatus = taskTrackerManager.getClusterStatus(); final int numTaskTrackers = clusterStatus.getTaskTrackers(); final int clusterMapCapacity = clusterStatus.getMaxMapTasks(); final int clusterReduceCapacity = clusterStatus.getMaxReduceTasks(); Collection<JobInProgress> jobQueue = jobQueueJobInProgressListener.getJobQueue(); // // Get map + reduce counts for the current tracker. // final int trackerMapCapacity = taskTrackerStatus.getMaxMapSlots(); final int trackerReduceCapacity = taskTrackerStatus.getMaxReduceSlots(); final int trackerRunningMaps = taskTrackerStatus.countMapTasks(); final int trackerRunningReduces = taskTrackerStatus.countReduceTasks(); // Assigned tasks List<Task> assignedTasks = new ArrayList<Task>(); // // Compute (running + pending) map and reduce task numbers across pool // int remainingReduceLoad = 0; int remainingMapLoad = 0; synchronized (jobQueue) { for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() == JobStatus.RUNNING) { remainingMapLoad += (job.desiredMaps() - job.finishedMaps()); if (job.scheduleReduces()) { remainingReduceLoad += (job.desiredReduces() - job.finishedReduces()); } } } } // Compute the 'load factor' for maps and reduces double mapLoadFactor = 0.0; if (clusterMapCapacity > 0) { mapLoadFactor = (double) remainingMapLoad / clusterMapCapacity; } double reduceLoadFactor = 0.0; if (clusterReduceCapacity > 0) { reduceLoadFactor = (double) remainingReduceLoad / clusterReduceCapacity; } // // In the below steps, we allocate first map tasks (if appropriate), // and then reduce tasks if appropriate. We go through all jobs // in order of job arrival; jobs only get serviced if their // predecessors are serviced, too. // // // We assign tasks to the current taskTracker if the given machine // has a workload that's less than the maximum load of that kind of // task. // However, if the cluster is close to getting loaded i.e. we don't // have enough _padding_ for speculative executions etc., we only // schedule the "highest priority" task i.e. the task from the job // with the highest priority. // final int trackerCurrentMapCapacity = Math.min((int) Math.ceil(mapLoadFactor * trackerMapCapacity), trackerMapCapacity); int availableMapSlots = trackerCurrentMapCapacity - trackerRunningMaps; boolean exceededMapPadding = false; if (availableMapSlots > 0) { exceededMapPadding = exceededPadding(true, clusterStatus, trackerMapCapacity); } int numLocalMaps = 0; int numNonLocalMaps = 0; boolean newIterationJob = false; scheduleMaps: for (int i = 0; i < availableMapSlots; ++i) { synchronized (jobQueue) { for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() != JobStatus.RUNNING) { continue; } if (job.getJobConf().isIterative()) { String iterativeAppID = job.getJobConf().getIterativeAlgorithmID(); if (iterativeAppID.equals("none")) { throw new IOException("please specify the iteration ID!"); } String jointype = job.getJobConf().get("mapred.iterative.jointype"); // prepare the iterationid map and jobtask map if (!this.tracker_mtask_map.containsKey(iterativeAppID)) { // a new iterative algorithm Map<String, LinkedList<Integer>> new_tracker_task_map = new HashMap<String, LinkedList<Integer>>(); this.tracker_mtask_map.put(iterativeAppID, new_tracker_task_map); Map<String, LinkedList<Integer>> new_tracker_rtask_map = new HashMap<String, LinkedList<Integer>>(); this.tracker_rtask_map.put(iterativeAppID, new_tracker_rtask_map); // record the first job of the series of jobs in the iterations this.first_job_map.put(iterativeAppID, job.getJobID()); // record the list of jobs for a iteration HashSet<JobID> jobs = new HashSet<JobID>(); jobs.add(job.getJobID()); this.iteration_jobs_map.put(iterativeAppID, jobs); } // this is the first job of the series of jobs if (this.first_job_map.get(iterativeAppID).equals(job.getJobID()) && job.getJobConf().isIterative()) { LOG.info(job.getJobID() + " is the first iteration job"); newIterationJob = true; } // this is one of the following jobs, and prepare a assignment list for the assignment if (!newIterationJob) { LOG.info(job.getJobID() + " is not the first iteration job"); this.iteration_jobs_map.get(iterativeAppID).add(job.getJobID()); if (this.mtask_assign_map.get(job.getJobID()) == null) { // prepare the map task assignment list LOG.info("for job " + job.getJobID() + "'s assignment:"); Map<String, LinkedList<Integer>> map_task_assign = new HashMap<String, LinkedList<Integer>>(); for (Map.Entry<String, LinkedList<Integer>> entry : this.tracker_mtask_map.get(iterativeAppID).entrySet()) { String tracker = entry.getKey(); LinkedList<Integer> taskids = entry.getValue(); LinkedList<Integer> copytaskids = new LinkedList<Integer>(); LOG.info("assign on tracker " + tracker); for (int taskid : taskids) { copytaskids.add(taskid); LOG.info("task id " + taskid); } map_task_assign.put(tracker, copytaskids); } this.mtask_assign_map.put(job.getJobID(), map_task_assign); // if one2one copy the map assign to reduce assign, the are with the same mapping if (jointype.equals("one2one")) { // prepare the reduce task assignment list Map<String, LinkedList<Integer>> reduce_task_assign = new HashMap<String, LinkedList<Integer>>(); for (Map.Entry<String, LinkedList<Integer>> entry : this.tracker_mtask_map.get(iterativeAppID).entrySet()) { String tracker = entry.getKey(); LinkedList<Integer> taskids = entry.getValue(); LinkedList<Integer> copytaskids = new LinkedList<Integer>(); for (int taskid : taskids) { copytaskids.add(taskid); } reduce_task_assign.put(tracker, copytaskids); } this.tracker_rtask_map.put(iterativeAppID, reduce_task_assign); } // prepare the reduce task assignment list for all cases Map<String, LinkedList<Integer>> reduce_task_assign = new HashMap<String, LinkedList<Integer>>(); for (Map.Entry<String, LinkedList<Integer>> entry : this.tracker_rtask_map.get(iterativeAppID).entrySet()) { String tracker = entry.getKey(); LinkedList<Integer> taskids = entry.getValue(); LinkedList<Integer> copytaskids = new LinkedList<Integer>(); for (int taskid : taskids) { copytaskids.add(taskid); } reduce_task_assign.put(tracker, copytaskids); } this.rtask_assign_map.put(job.getJobID(), reduce_task_assign); } } Task t = null; // the first iteration or following iteration // if the first iteration: assign taskid by default (exception for the one2mul case, // where we assign staring from 0,...,n) // else if the following iterations: assign taskid based on the first iteration // assignment if (newIterationJob) { /** * the one2mul case should be carefully taken care, we want to assgin map0,map1,map2 * and reduce0 to a tracker, and assign map3,map4,map5 and reduce1 to another tracker */ if (jointype.equals("one2mul") && !tracker_rtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { // if contain the tracker, that means we have assigned tasks for this tracker int scala = job.getJobConf().getInt("mapred.iterative.data.scala", 1); // int mapsEachTracker = job.getJobConf().getNumMapTasks() / numTaskTrackers; int reducersEachTracker = job.getJobConf().getNumReduceTasks() / numTaskTrackers; if (job.getJobConf().getNumReduceTasks() % numTaskTrackers != 0) throw new IOException( "job.getJobConf().getNumReduceTasks() % numTaskTrackers != 0"); if (!this.tracker_mtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> tasklist = new LinkedList<Integer>(); this.tracker_mtask_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), tasklist); } if (!this.tracker_rtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> tasklist = new LinkedList<Integer>(); this.tracker_rtask_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), tasklist); } // for debugging String debugout1 = "maps: "; String debugout2 = "reduces: "; int reduceOffsetId = (tracker_rtask_map.get(iterativeAppID).size() - 1) * reducersEachTracker; // the start reduce id for (int count = 0; count < reducersEachTracker; count++) { int reducepartitionid = reduceOffsetId + count; debugout2 += reducepartitionid + " "; tracker_rtask_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(reducepartitionid); for (int count2 = 0; count2 < scala; count2++) { int mappartitionid = reducepartitionid * scala + count2; // int mapid = job.splitTaskMap.get(mappartitionid); debugout1 += mappartitionid + " "; this.tracker_mtask_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(mappartitionid); } } // print out for debug LOG.info( "tracker " + taskTracker.getTrackerName() + " assigned tasks " + debugout1 + " and " + debugout2); // make the assignment list String tracker = taskTracker.getTrackerName(); LinkedList<Integer> mtaskids = this.tracker_mtask_map.get(iterativeAppID).get(taskTracker.getTrackerName()); LinkedList<Integer> mcopytaskids = new LinkedList<Integer>(); for (int taskid : mtaskids) { mcopytaskids.add(taskid); } if (!mtask_assign_map.containsKey(job.getJobID())) { Map<String, LinkedList<Integer>> map_task_assign = new HashMap<String, LinkedList<Integer>>(); this.mtask_assign_map.put(job.getJobID(), map_task_assign); } this.mtask_assign_map.get(job.getJobID()).put(tracker, mcopytaskids); // prepare the reduce task assignment list LinkedList<Integer> rtaskids = this.tracker_rtask_map.get(iterativeAppID).get(taskTracker.getTrackerName()); LinkedList<Integer> rcopytaskids = new LinkedList<Integer>(); for (int taskid : rtaskids) { rcopytaskids.add(taskid); } if (!rtask_assign_map.containsKey(job.getJobID())) { Map<String, LinkedList<Integer>> reduce_task_assign = new HashMap<String, LinkedList<Integer>>(); this.rtask_assign_map.put(job.getJobID(), reduce_task_assign); } this.rtask_assign_map.get(job.getJobID()).put(tracker, rcopytaskids); // assign a map task for this tracker Integer target = null; try { target = this.mtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .peekFirst(); } catch (Exception e) { e.printStackTrace(); } if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all map tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewNodeOrRackLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } } else { t = job.obtainNewNodeOrRackLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); } } else { Integer target = null; try { target = this.mtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .peekFirst(); } catch (Exception e) { e.printStackTrace(); } if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all map tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewNodeOrRackLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } } if (t != null) { assignedTasks.add(t); ++numLocalMaps; // new iteration job and the first task for a tasktracker // for one2mul case, we don't need to record the assignment, since we already made the // assignment list beforehand if (!newIterationJob || jointype.equals("one2mul")) { // poll, remove this.mtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .pollFirst(); LOG.info("assigning task " + t.getTaskID() + " on " + taskTracker.getTrackerName()); } else { // record the assignment list for map tasks if (!this.tracker_mtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> tasklist = new LinkedList<Integer>(); this.tracker_mtask_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), tasklist); } this.tracker_mtask_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(t.getTaskID().getTaskID().getId()); // prepare the reduce assignment, for mapping with reduce if (jointype.equals("one2one")) { // prepare the reduce assignment, for mapping with reduce if (!first_job_reduces_map.containsKey(iterativeAppID)) { Map<String, LinkedList<Integer>> tracker_reduce_map = new HashMap<String, LinkedList<Integer>>(); first_job_reduces_map.put(iterativeAppID, tracker_reduce_map); } if (!first_job_reduces_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> reduces = new LinkedList<Integer>(); first_job_reduces_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), reduces); } first_job_reduces_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(t.getTaskID().getTaskID().getId()); } LOG.info("assigning task " + t.getTaskID() + " on " + taskTracker.getTrackerName()); } // Don't assign map tasks to the hilt! // Leave some free slots in the cluster for future task-failures, // speculative tasks etc. beyond the highest priority job if (exceededMapPadding) { break scheduleMaps; } // Try all jobs again for the next Map task break; } LOG.error("New Node Or Rack Local Map Task failed!"); if (newIterationJob) { // Try to schedule a node-local or rack-local Map task t = job.obtainNewNonLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); } else { Integer target = this.mtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .peekFirst(); if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all map tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewNonLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } } if (t != null) { assignedTasks.add(t); ++numNonLocalMaps; // new iteration job and the first task for a tasktracker if (newIterationJob) { if (!this.tracker_mtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> tasklist = new LinkedList<Integer>(); this.tracker_mtask_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), tasklist); } this.tracker_mtask_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(t.getTaskID().getTaskID().getId()); } else { // poll, remove this.mtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .pollFirst(); } // We assign at most 1 off-switch or speculative task // This is to prevent TaskTrackers from stealing local-tasks // from other TaskTrackers. break scheduleMaps; } } else { // not an iterative algorithm, normal schedule Task t = null; // Try to schedule a node-local or rack-local Map task t = job.obtainNewNodeOrRackLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { assignedTasks.add(t); ++numLocalMaps; // Don't assign map tasks to the hilt! // Leave some free slots in the cluster for future task-failures, // speculative tasks etc. beyond the highest priority job if (exceededMapPadding) { break scheduleMaps; } // Try all jobs again for the next Map task break; } // Try to schedule a node-local or rack-local Map task t = job.obtainNewNonLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { assignedTasks.add(t); ++numNonLocalMaps; // We assign at most 1 off-switch or speculative task // This is to prevent TaskTrackers from stealing local-tasks // from other TaskTrackers. break scheduleMaps; } } } } } int assignedMaps = assignedTasks.size(); // // Same thing, but for reduce tasks // However we _never_ assign more than 1 reduce task per heartbeat // /** should maintain the reduce task location for the termination check */ final int trackerCurrentReduceCapacity = Math.min((int) Math.ceil(reduceLoadFactor * trackerReduceCapacity), trackerReduceCapacity); final int availableReduceSlots = Math.min((trackerCurrentReduceCapacity - trackerRunningReduces), 1); boolean exceededReducePadding = false; // LOG.info("availableReduceSlots " + availableReduceSlots); if (availableReduceSlots > 0) { exceededReducePadding = exceededPadding(false, clusterStatus, trackerReduceCapacity); synchronized (jobQueue) { for (JobInProgress job : jobQueue) { LOG.info("job " + job.getJobID()); if (job.getStatus().getRunState() != JobStatus.RUNNING || job.numReduceTasks == 0) { LOG.info("have to continue " + job.getStatus().getRunState()); continue; } Task t = null; if (job.getJobConf().isIterative()) { String iterativeAppID = job.getJobConf().getIterativeAlgorithmID(); if (iterativeAppID.equals("none")) { throw new IOException("please specify the iteration ID!"); } String jointype = job.getJobConf().get("mapred.iterative.jointype"); if (jointype.equals("one2one")) { // one-to-one or one-to-mul jobs if (this.first_job_map.get(iterativeAppID).equals(job.getJobID()) && job.getJobConf().isIterative()) { LOG.info(job.getJobID() + " is the first iteration job for reduce"); newIterationJob = true; } Integer target = null; if (newIterationJob) { if (first_job_reduces_map.get(iterativeAppID) == null) { throw new IOException( "I think something is wrong since the tasktracker never receive " + "a map task with iterativeapp id " + iterativeAppID); } if (first_job_reduces_map.get(iterativeAppID).get(taskTracker.getTrackerName()) == null) { throw new IOException( "I think something is wrong since the tasktracker never receive " + "a map task with iterativeapp id " + iterativeAppID + " from " + taskTracker.getTrackerName()); } target = this.first_job_reduces_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .pollFirst(); } else { // the task assignment has already been processed during the map task assignment, so // never use tracker_rtask_map target = this.rtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .pollFirst(); } if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all reduce tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } } else if (jointype.equals("one2mul")) { Integer target = this.rtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .pollFirst(); if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all reduce tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } } else { // one-to-all case, assign tasks in the first iteration job, and remember this mapping // this is the first job of the series of jobs if (this.first_job_map.get(iterativeAppID).equals(job.getJobID())) { LOG.info(job.getJobID() + " is the first iteration job for reduce"); newIterationJob = true; } /* //this is one of the following jobs, and prepare a assignment list for the assignment else{ LOG.info(job.getJobID() + " is not the first iteration job for reduce"); if(this.rtask_assign_map.get(job.getJobID()) == null){ //prepare the map task assignment list Map<String, LinkedList<Integer>> reduce_task_assign = new HashMap<String, LinkedList<Integer>>(); for(Map.Entry<String, LinkedList<Integer>> entry : this.tracker_rtask_map.get(iterativeAppID).entrySet()){ String tracker = entry.getKey(); LinkedList<Integer> taskids = entry.getValue(); LinkedList<Integer> copytaskids = new LinkedList<Integer>(); for(int taskid : taskids){ copytaskids.add(taskid); } reduce_task_assign.put(tracker, copytaskids); } this.rtask_assign_map.put(job.getJobID(), reduce_task_assign); } } */ // the first iteration or following iteration // if the first iteration: assign taskid by default // else if the following iterations: assign taskid based on the first iteration // assignment if (newIterationJob) { t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { if (!this.tracker_rtask_map .get(iterativeAppID) .containsKey(taskTracker.getTrackerName())) { LinkedList<Integer> tasklist = new LinkedList<Integer>(); this.tracker_rtask_map .get(iterativeAppID) .put(taskTracker.getTrackerName(), tasklist); } this.tracker_rtask_map .get(iterativeAppID) .get(taskTracker.getTrackerName()) .add(t.getTaskID().getTaskID().getId()); LOG.info( "assigning reduce task " + t.getTaskID() + " on " + taskTracker.getTrackerName()); } } else { Integer target = this.rtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .peekFirst(); if (target == null) { // all have been assigned, no more work, maybe it should help others to process LOG.info( "all map tasks on tasktracker " + taskTracker.getTrackerName() + " have been processed"); break; } else { t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts(), target); } if (t != null) { // poll, remove this.rtask_assign_map .get(job.getJobID()) .get(taskTracker.getTrackerName()) .pollFirst(); LOG.info( "assigning reduce task " + t.getTaskID() + " on " + taskTracker.getTrackerName()); } } } } else { t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); } LOG.info("try to assign new task " + t); if (t != null) { assignedTasks.add(t); break; } // Don't assign reduce tasks to the hilt! // Leave some free slots in the cluster for future task-failures, // speculative tasks etc. beyond the highest priority job if (exceededReducePadding) { break; } } } } if (LOG.isDebugEnabled()) { LOG.debug( "Task assignments for " + taskTrackerStatus.getTrackerName() + " --> " + "[" + mapLoadFactor + ", " + trackerMapCapacity + ", " + trackerCurrentMapCapacity + ", " + trackerRunningMaps + "] -> [" + (trackerCurrentMapCapacity - trackerRunningMaps) + ", " + assignedMaps + " (" + numLocalMaps + ", " + numNonLocalMaps + ")] [" + reduceLoadFactor + ", " + trackerReduceCapacity + ", " + trackerCurrentReduceCapacity + "," + trackerRunningReduces + "] -> [" + (trackerCurrentReduceCapacity - trackerRunningReduces) + ", " + (assignedTasks.size() - assignedMaps) + "]"); } return assignedTasks; }
/** * Get the pool name for a JobInProgress from its configuration. This uses the value of * mapred.fairscheduler.pool if specified, otherwise the value of the property named in * mapred.fairscheduler.poolnameproperty if that is specified. Otherwise if neither is specified * it uses the "user.name" property in the jobconf by default. */ public String getPoolName(JobInProgress job) { Configuration conf = job.getJobConf(); return conf.get(EXPLICIT_POOL_PROPERTY, conf.get(poolNameProperty, Pool.DEFAULT_POOL_NAME)) .trim(); }
/** Change the pool of a particular job */ public synchronized void setPool(JobInProgress job, String pool) { removeJob(job); job.getJobConf().set(EXPLICIT_POOL_PROPERTY, pool); addJob(job); }
/* * TODO: * For Elf: need to change the major schedule logic, scheduling need * to be *datacenter-aware* * */ @Override public synchronized List<Task> assignTasks(TaskTracker taskTracker) throws IOException { TaskTrackerStatus taskTrackerStatus = taskTracker.getStatus(); ClusterStatus clusterStatus = taskTrackerManager.getClusterStatus(); final int numTaskTrackers = clusterStatus.getTaskTrackers(); final int clusterMapCapacity = clusterStatus.getMaxMapTasks(); final int clusterReduceCapacity = clusterStatus.getMaxReduceTasks(); Collection<JobInProgress> jobQueue = jobQueueJobInProgressListener.getJobQueue(); // // Get map + reduce counts for the current tracker. // final int trackerMapCapacity = taskTrackerStatus.getMaxMapSlots(); final int trackerReduceCapacity = taskTrackerStatus.getMaxReduceSlots(); final int trackerRunningMaps = taskTrackerStatus.countMapTasks(); final int trackerRunningReduces = taskTrackerStatus.countReduceTasks(); // Assigned tasks List<Task> assignedTasks = new ArrayList<Task>(); // // Compute (running + pending) map and reduce task numbers across pool // int remainingReduceLoad = 0; int remainingMapLoad = 0; synchronized (jobQueue) { for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() == JobStatus.RUNNING) { remainingMapLoad += (job.desiredMaps() - job.finishedMaps()); if (job.scheduleReduces()) { remainingReduceLoad += (job.desiredReduces() - job.finishedReduces()); } } } } // Compute the 'load factor' for maps and reduces double mapLoadFactor = 0.0; if (clusterMapCapacity > 0) { mapLoadFactor = (double) remainingMapLoad / clusterMapCapacity; } double reduceLoadFactor = 0.0; if (clusterReduceCapacity > 0) { reduceLoadFactor = (double) remainingReduceLoad / clusterReduceCapacity; } // // In the below steps, we allocate first map tasks (if appropriate), // and then reduce tasks if appropriate. We go through all jobs // in order of job arrival; jobs only get serviced if their // predecessors are serviced, too. // // // We assign tasks to the current taskTracker if the given machine // has a workload that's less than the maximum load of that kind of // task. // However, if the cluster is close to getting loaded i.e. we don't // have enough _padding_ for speculative executions etc., we only // schedule the "highest priority" task i.e. the task from the job // with the highest priority. // final int trackerCurrentMapCapacity = Math.min((int) Math.ceil(mapLoadFactor * trackerMapCapacity), trackerMapCapacity); int availableMapSlots = trackerCurrentMapCapacity - trackerRunningMaps; boolean exceededMapPadding = false; if (availableMapSlots > 0) { exceededMapPadding = exceededPadding(true, clusterStatus, trackerMapCapacity); } int numLocalMaps = 0; int numNonLocalMaps = 0; scheduleMaps: // TODO: for Elf // The main schedule logic here, outer for loop is for every slot, inner loop is for each job for (int i = 0; i < availableMapSlots; ++i) { synchronized (jobQueue) { for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() != JobStatus.RUNNING) { continue; } Task t = null; // Try to schedule a node-local or rack-local Map task t = job.obtainNewLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { assignedTasks.add(t); ++numLocalMaps; // Don't assign map tasks to the hilt! // Leave some free slots in the cluster for future task-failures, // speculative tasks etc. beyond the highest priority job if (exceededMapPadding) { break scheduleMaps; } // Try all jobs again for the next Map task // Note: it's FIFO here: next time in the inner for loop the head-of-queue // will still be chosen break; } // If no locality for this job, try launching non-local // Try to schedule a node-local or rack-local Map task --> original comments // FIXME: is the above comment correct? seems should be non-local task t = job.obtainNewNonLocalMapTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { assignedTasks.add(t); ++numNonLocalMaps; // We assign at most 1 off-switch or speculative task // This is to prevent TaskTrackers from stealing local-tasks // from other TaskTrackers. break scheduleMaps; } } } } int assignedMaps = assignedTasks.size(); // // Same thing, but for reduce tasks // However we _never_ assign more than 1 reduce task per heartbeat // final int trackerCurrentReduceCapacity = Math.min((int) Math.ceil(reduceLoadFactor * trackerReduceCapacity), trackerReduceCapacity); final int availableReduceSlots = Math.min((trackerCurrentReduceCapacity - trackerRunningReduces), 1); boolean exceededReducePadding = false; if (availableReduceSlots > 0) { exceededReducePadding = exceededPadding(false, clusterStatus, trackerReduceCapacity); synchronized (jobQueue) { for (JobInProgress job : jobQueue) { if (job.getStatus().getRunState() != JobStatus.RUNNING || job.numReduceTasks == 0) { continue; } Task t = job.obtainNewReduceTask( taskTrackerStatus, numTaskTrackers, taskTrackerManager.getNumberOfUniqueHosts()); if (t != null) { assignedTasks.add(t); break; } // Don't assign reduce tasks to the hilt! // Leave some free slots in the cluster for future task-failures, // speculative tasks etc. beyond the highest priority job if (exceededReducePadding) { break; } } } } if (LOG.isDebugEnabled()) { LOG.debug( "Task assignments for " + taskTrackerStatus.getTrackerName() + " --> " + "[" + mapLoadFactor + ", " + trackerMapCapacity + ", " + trackerCurrentMapCapacity + ", " + trackerRunningMaps + "] -> [" + (trackerCurrentMapCapacity - trackerRunningMaps) + ", " + assignedMaps + " (" + numLocalMaps + ", " + numNonLocalMaps + ")] [" + reduceLoadFactor + ", " + trackerReduceCapacity + ", " + trackerCurrentReduceCapacity + "," + trackerRunningReduces + "] -> [" + (trackerCurrentReduceCapacity - trackerRunningReduces) + ", " + (assignedTasks.size() - assignedMaps) + "]"); } return assignedTasks; }
/** Check refreshNodes for decommissioning blacklisted nodes. */ public void testBlacklistedNodeDecommissioning() throws Exception { LOG.info("Testing blacklisted node decommissioning"); MiniMRCluster mr = null; JobTracker jt = null; try { // start mini mr JobConf jtConf = new JobConf(); jtConf.set("mapred.max.tracker.blacklists", "1"); mr = new MiniMRCluster(0, 0, 2, "file:///", 1, null, null, null, jtConf); jt = mr.getJobTrackerRunner().getJobTracker(); assertEquals("Trackers not up", 2, jt.taskTrackers().size()); // validate the total tracker count assertEquals( "Active tracker count mismatch", 2, jt.getClusterStatus(false).getTaskTrackers()); // validate blacklisted count assertEquals( "Blacklisted tracker count mismatch", 0, jt.getClusterStatus(false).getBlacklistedTrackers()); // run a failing job to blacklist the tracker JobConf jConf = mr.createJobConf(); jConf.set("mapred.max.tracker.failures", "1"); jConf.setJobName("test-job-fail-once"); jConf.setMapperClass(FailOnceMapper.class); jConf.setReducerClass(IdentityReducer.class); jConf.setNumMapTasks(1); jConf.setNumReduceTasks(0); RunningJob job = UtilsForTests.runJob(jConf, new Path(TEST_DIR, "in"), new Path(TEST_DIR, "out")); job.waitForCompletion(); // validate the total tracker count assertEquals( "Active tracker count mismatch", 1, jt.getClusterStatus(false).getTaskTrackers()); // validate blacklisted count assertEquals( "Blacklisted tracker count mismatch", 1, jt.getClusterStatus(false).getBlacklistedTrackers()); // find the blacklisted tracker String trackerName = null; for (TaskTrackerStatus status : jt.taskTrackers()) { if (jt.isBlacklisted(status.getTrackerName())) { trackerName = status.getTrackerName(); break; } } // get the hostname String hostToDecommission = JobInProgress.convertTrackerNameToHostName(trackerName); LOG.info("Decommissioning tracker " + hostToDecommission); // decommission the node HashSet<String> decom = new HashSet<String>(1); decom.add(hostToDecommission); jt.decommissionNodes(decom); // validate // check the cluster status and tracker size assertEquals( "Tracker is not lost upon host decommissioning", 1, jt.getClusterStatus(false).getTaskTrackers()); assertEquals( "Blacklisted tracker count incorrect in cluster status " + "after decommissioning", 0, jt.getClusterStatus(false).getBlacklistedTrackers()); assertEquals("Tracker is not lost upon host decommissioning", 1, jt.taskTrackers().size()); } finally { if (mr != null) { mr.shutdown(); mr = null; jt = null; FileUtil.fullyDelete(new File(TEST_DIR.toString())); } } }
/** * Initialize a job. * * @param job required initialzied. */ public void initJob(JobInProgress job) throws IOException { job.initStaffs(); }