@Override public void start() { try { Configuration conf = getConf(); eagerInitListener.start(); taskTrackerManager.addJobInProgressListener(eagerInitListener); taskTrackerManager.addJobInProgressListener(jobListener); poolMgr = new PoolManager(conf); loadMgr = (LoadManager) ReflectionUtils.newInstance( conf.getClass( "mapred.fairscheduler.loadmanager", CapBasedLoadManager.class, LoadManager.class), conf); loadMgr.setTaskTrackerManager(taskTrackerManager); loadMgr.start(); taskSelector = (TaskSelector) ReflectionUtils.newInstance( conf.getClass( "mapred.fairscheduler.taskselector", DefaultTaskSelector.class, TaskSelector.class), conf); taskSelector.setTaskTrackerManager(taskTrackerManager); taskSelector.start(); Class<?> weightAdjClass = conf.getClass("mapred.fairscheduler.weightadjuster", null); if (weightAdjClass != null) { weightAdjuster = (WeightAdjuster) ReflectionUtils.newInstance(weightAdjClass, conf); } assignMultiple = conf.getBoolean("mapred.fairscheduler.assignmultiple", false); sizeBasedWeight = conf.getBoolean("mapred.fairscheduler.sizebasedweight", false); initialized = true; running = true; lastUpdateTime = clock.getTime(); // Start a thread to update deficits every UPDATE_INTERVAL if (runBackgroundUpdates) new UpdateThread().start(); // Register servlet with JobTracker's Jetty server if (taskTrackerManager instanceof JobTracker) { JobTracker jobTracker = (JobTracker) taskTrackerManager; StatusHttpServer infoServer = jobTracker.infoServer; infoServer.setAttribute("scheduler", this); infoServer.addServlet("scheduler", "/scheduler", FairSchedulerServlet.class); } } catch (Exception e) { // Can't load one of the managers - crash the JobTracker now while it is // starting up so that the user notices. throw new RuntimeException("Failed to start FairScheduler", e); } LOG.info("Successfully configured FairScheduler"); }
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; } } }
@Override public synchronized List<Task> assignTasks(TaskTrackerStatus tracker) throws IOException { if (!initialized) // Don't try to assign tasks if we haven't yet started up return null; // Reload allocations file if it hasn't been loaded in a while poolMgr.reloadAllocsIfNecessary(); // Compute total runnable maps and reduces int runnableMaps = 0; int runnableReduces = 0; for (JobInProgress job : infos.keySet()) { runnableMaps += runnableTasks(job, TaskType.MAP); runnableReduces += runnableTasks(job, TaskType.REDUCE); } // Compute total map/reduce slots // In the future we can precompute this if the Scheduler becomes a // listener of tracker join/leave events. int totalMapSlots = getTotalSlots(TaskType.MAP); int totalReduceSlots = getTotalSlots(TaskType.REDUCE); // Scan to see whether any job needs to run a map, then a reduce ArrayList<Task> tasks = new ArrayList<Task>(); TaskType[] types = new TaskType[] {TaskType.MAP, TaskType.REDUCE}; for (TaskType taskType : types) { boolean canAssign = (taskType == TaskType.MAP) ? loadMgr.canAssignMap(tracker, runnableMaps, totalMapSlots) : loadMgr.canAssignReduce(tracker, runnableReduces, totalReduceSlots); if (canAssign) { // Figure out the jobs that need this type of task List<JobInProgress> candidates = new ArrayList<JobInProgress>(); for (JobInProgress job : infos.keySet()) { if (job.getStatus().getRunState() == JobStatus.RUNNING && neededTasks(job, taskType) > 0) { candidates.add(job); } } // Sort jobs by deficit (for Fair Sharing) or submit time (for FIFO) Comparator<JobInProgress> comparator = useFifo ? new FifoJobComparator() : new DeficitComparator(taskType); Collections.sort(candidates, comparator); for (JobInProgress job : candidates) { Task task = (taskType == TaskType.MAP ? taskSelector.obtainNewMapTask(tracker, job) : taskSelector.obtainNewReduceTask(tracker, job)); if (task != null) { // Update the JobInfo for this job so we account for the launched // tasks during this update interval and don't try to launch more // tasks than the job needed on future heartbeats JobInfo info = infos.get(job); if (taskType == TaskType.MAP) { info.runningMaps++; info.neededMaps--; } else { info.runningReduces++; info.neededReduces--; } tasks.add(task); if (!assignMultiple) return tasks; break; } } } } // If no tasks were found, return null return tasks.isEmpty() ? null : tasks; }