Java tutorial
/* * Copyright 2012 Eurecom (http://www.eurecom.fr) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.hadoop.mapred; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.hadoop.mapred.HFSPScheduler.QueueType; import org.apache.hadoop.mapreduce.TaskType; public class AssignTasksHelper { enum Phase { TRAIN, SIZE_BASED, FALLBACK_TRAIN, FALLBACK_SIZE_BASED } static class TaskStatuses { TreeMap<TaskAttemptID, TaskStatus> taskStatuses; TreeMap<TaskAttemptID, TaskStatus> suspendedTaskStatuses; StringBuilder builder = new StringBuilder(); public TaskStatuses() { this(null, null); } public TaskStatuses(Comparator<TaskAttemptID> tasksComparator, Comparator<TaskAttemptID> suspendedTasksComparator) { this.taskStatuses = new TreeMap<TaskAttemptID, TaskStatus>(tasksComparator); this.suspendedTaskStatuses = new TreeMap<TaskAttemptID, TaskStatus>(suspendedTasksComparator); } // public TaskStatus put(TaskStatus status) { // if (status.getRunState() == TaskStatus.State.SUSPENDED) // return this.suspendedTaskStatuses.put(status.getTaskID(), status); // else // return this.taskStatuses.put(status.getTaskID(), status); // } @Override public String toString() { builder.delete(0, builder.length()); builder.append("TaskStatuses:{ ").append(" statusesTAID: ["); boolean first = true; for (Entry<TaskAttemptID, TaskStatus> entry : this.taskStatuses.entrySet()) { if (first) first = false; else builder.append(", "); builder.append(entry.getKey()); } builder.append("], suspendedStatusesTAID: ["); first = true; for (Entry<TaskAttemptID, TaskStatus> entry : this.suspendedTaskStatuses.entrySet()) { if (first) first = false; else builder.append(", "); builder.append(entry.getKey()); } builder.append("]}"); return builder.toString(); } } /** * This is the helper that manage the assignTasks for tasks of a particular * type */ static class HelperForType { final TaskType taskType; int maxTrainSlots; int maxSizeBasedSlots; int numTrainTasksForJob; int startAvailableSlots; int currAvailableSlots; public int startRunningTT; public int startRunningSBT; int runningTrainTasks; int runningSizeBasedTasks; boolean doTrainScheduling; boolean doSizeBasedScheduling; Map<Phase, LinkedList<ActionLog>> actions = new HashMap<Phase, LinkedList<ActionLog>>(); TreeMap<JobDurationInfo, TaskStatuses> taskStatusesSizeBased = new TreeMap<JobDurationInfo, TaskStatuses>( Collections.reverseOrder(HFSPScheduler.JOB_DURATION_COMPARATOR)); public HelperForType(TaskType type) { this.taskType = type; for (Phase phase : Phase.values()) { this.actions.put(phase, new LinkedList<ActionLog>()); } } public void clear() { for (Phase phase : Phase.values()) { this.actions.get(phase).clear(); } this.taskStatusesSizeBased.clear(); } public boolean canAssignTrain() { return (!this.doSizeBasedScheduling || this.runningTrainTasks < this.maxTrainSlots); } public boolean tooManySizeBasedTasks() { return this.runningSizeBasedTasks > this.maxSizeBasedSlots; } /** Total available slots in the cluster for training */ public int totAvailableTrainSlots() { if (this.doTrainScheduling && this.doSizeBasedScheduling) return this.maxTrainSlots - this.runningTrainTasks; if (this.doTrainScheduling) return this.maxTrainSlots + this.maxSizeBasedSlots - this.runningTrainTasks; return 0; } /** Total available slots in the cluster for size based scheduling */ public int totAvailableSizeBasedSlots() { if (this.doTrainScheduling && this.doSizeBasedScheduling) return this.maxSizeBasedSlots - this.runningSizeBasedTasks; if (this.doSizeBasedScheduling) { int availSBSlots = this.maxTrainSlots + this.maxSizeBasedSlots - this.runningSizeBasedTasks; HFSPScheduler.LOG.info("totAvailableSizeBasedSlots: " + "maxTrainSlots: " + this.maxTrainSlots + " maxSizeBasedSlots: " + this.maxSizeBasedSlots + " runningSizeBasedTasks: " + this.runningSizeBasedTasks + " => " + availSBSlots); return availSBSlots; } return 0; } } private HFSPScheduler scheduler; LinkedList<Task> result = new LinkedList<Task>(); TaskTrackerStatus ttStatus = null; HelperForType mapHelper = new HelperForType(TaskType.MAP); HelperForType reduceHelper = new HelperForType(TaskType.REDUCE); public long currentTime; HelperForType helper(TaskType type) { return type == TaskType.MAP ? this.mapHelper : this.reduceHelper; } public AssignTasksHelper(HFSPScheduler scheduler) { this.scheduler = scheduler; } public void init(TaskTrackerStatus ttStatus) { this.result.clear(); this.ttStatus = ttStatus; this.currentTime = scheduler.getClock().getTime(); for (TaskType type : HFSPScheduler.TASK_TYPES) this.helper(type).clear(); this.initTaskStatuses(); for (TaskType type : HFSPScheduler.TASK_TYPES) { boolean isMap = type == TaskType.MAP; HelperForType helper = this.helper(type); this.setStartAvailableSlots(type, isMap ? ttStatus.getAvailableMapSlots() : ttStatus.getAvailableReduceSlots()); helper.runningTrainTasks = scheduler.getNumRunningTasksNotSuspended(QueueType.TRAIN, type); helper.startRunningTT = helper.runningTrainTasks; helper.runningSizeBasedTasks = scheduler.getNumRunningTasksNotSuspended(QueueType.SIZE_BASED, type); helper.startRunningSBT = helper.runningSizeBasedTasks; helper.doTrainScheduling = scheduler.isSchedulingActive(QueueType.TRAIN, type); helper.doSizeBasedScheduling = scheduler.isSchedulingActive(QueueType.SIZE_BASED, type); // max slots allocated to training and for size based scheduling helper.maxTrainSlots = isMap ? scheduler.numSlotsForMapTrain : scheduler.numSlotsForReduceTrain; helper.maxSizeBasedSlots = scheduler.getMaxTasks(type) - helper.maxTrainSlots; // num slots allocated to single job helper.numTrainTasksForJob = isMap ? scheduler.numMapTrainSlotsForJob : scheduler.numReduceTrainSlotsForJob; } } private void setStartAvailableSlots(TaskType type, int i) { HelperForType at = this.helper(type); at.startAvailableSlots = i; at.currAvailableSlots = i; } public int getAvailableSlots(TaskType type) { return this.helper(type).currAvailableSlots; } TaskType getTaskType(TaskAttemptID tid) { return tid.getTaskID().isMap() ? TaskType.MAP : TaskType.REDUCE; } private void initTaskStatuses() { for (TaskStatus taskStatus : ttStatus.getTaskReports()) { TaskType taskType = this.getTaskType(taskStatus.getTaskID()); JobID jobID = taskStatus.getTaskID().getJobID(); if (null != scheduler.getJob(jobID, QueueType.SIZE_BASED, taskType)) { HelperForType assistant = this.helper(taskType); JobDurationInfo jdi = scheduler.getDuration(jobID, taskType); if (!assistant.taskStatusesSizeBased.containsKey(jdi)) { assistant.taskStatusesSizeBased.put(jdi, new TaskStatuses()); } TaskStatuses taskStatuses = assistant.taskStatusesSizeBased.get(jdi); if (this.scheduler.preemptionStrategy.isPreempted(taskStatus)) { taskStatuses.suspendedTaskStatuses.put(taskStatus.getTaskID(), taskStatus); } else { taskStatuses.taskStatuses.put(taskStatus.getTaskID(), taskStatus); } } } } /** * Log a suspension. Note that a suspended tasks will be considered from a * size based job */ public void suspend(TaskAttemptID taskID, JobID jID, Phase phase) { this.suspend(taskID, jID, phase, null, null); } /** * Log a suspension. Note that a suspended tasks will be considered from a * size based job */ public void suspend(TaskAttemptID taskID, JobID jID, Phase phase, JobDurationInfo jdiSuspended, JobDurationInfo jdiSuspender) { HelperForType helper = this.helper(this.getTaskType(taskID)); if (jdiSuspended == null) helper.actions.get(phase).add(new SuspendForTrainLog(taskID, jID)); else helper.actions.get(phase).add(new SuspendBiggerLog(taskID, jID, jdiSuspended.getPhaseDuration(), jdiSuspender.getPhaseDuration())); helper.currAvailableSlots++; helper.runningSizeBasedTasks--; } public void kill(TaskAttemptID pTAID, JobID jobID, Phase phase) { this.kill(pTAID, jobID, phase, null, null); } public void kill(TaskAttemptID pTAID, JobID jobID, Phase phase, JobDurationInfo jdiKilled, JobDurationInfo jdiKiller) { HelperForType helper = this.helper(this.getTaskType(pTAID)); if (jdiKilled == null) helper.actions.get(phase).add(new KillForTrainLog(pTAID, jobID)); else helper.actions.get(phase).add( new KillBiggerLog(pTAID, jobID, jdiKilled.getPhaseDuration(), jdiKiller.getPhaseDuration())); helper.currAvailableSlots++; helper.runningSizeBasedTasks--; } /** * Log a slot obtain. The phase is also used for internal counters */ public void slotObtained(Task task, Phase phase) { HelperForType helper = this.helper(task.isMapTask() ? TaskType.MAP : TaskType.REDUCE); helper.actions.get(phase).add(new AssignLog(task.getTaskID())); helper.currAvailableSlots--; if (phase == Phase.TRAIN || phase == Phase.FALLBACK_TRAIN) helper.runningTrainTasks++; else if (phase == Phase.SIZE_BASED || phase == Phase.FALLBACK_SIZE_BASED) helper.runningSizeBasedTasks++; this.result.add(task); } /** * Log a resume. Note that a suspended tasks will be considered from a size * based job */ void resume(TaskAttemptID tAID, Phase phase) { HelperForType helper = this.helper(this.getTaskType(tAID)); helper.actions.get(phase).add(new ResumeLog(tAID)); helper.currAvailableSlots--; helper.runningSizeBasedTasks++; } public void logInfos(Log log) { for (TaskType type : HFSPScheduler.TASK_TYPES) { this.logGeneralInfos(log, type); if (isSomethingChanged(this.helper(type))) { this.logChanges(log, type); } } } /** General informations used by the assignTasks */ private void logGeneralInfos(Log log, TaskType type) { HelperForType helper = this.helper(type); StringBuilder builder = new StringBuilder("assignTasks("); builder.append(this.ttStatus.getTrackerName()).append(":").append(type).append(") mode: "); if (helper.doSizeBasedScheduling) { builder.append(helper.doTrainScheduling ? "T+SB" : "SB"); } else { builder.append(helper.doTrainScheduling ? "T" : "None"); } if (helper.doSizeBasedScheduling || helper.doTrainScheduling) { builder.append(" maxT: " + helper.maxTrainSlots).append(" maxSB: " + helper.maxSizeBasedSlots) .append(" minTTasks: " + helper.numTrainTasksForJob) .append(" totAvailT: " + helper.totAvailableTrainSlots()) .append(" totAvailSB: " + helper.totAvailableSizeBasedSlots()); } log.debug(builder.toString()); } private void logChanges(Log log, TaskType type) { HelperForType helper = this.helper(type); StringBuilder builder = new StringBuilder("assignTasks("); builder.append(this.ttStatus.getTrackerName()).append(":").append(type).append("):") .append(" totRunningTrainTasks: " + helper.startRunningTT + " -> " + helper.runningTrainTasks) .append(" totRunningSizeBasedTasks: " + helper.startRunningSBT + " -> " + helper.runningSizeBasedTasks) .append(" tt.availableSlots: " + helper.startAvailableSlots + " -> " + helper.currAvailableSlots) .append(" actions: " + helper.actions); log.debug(builder.toString()); } private boolean isSomethingChanged(HelperForType helper) { return !helper.actions.get(Phase.TRAIN).isEmpty() || !helper.actions.get(Phase.SIZE_BASED).isEmpty() || !helper.actions.get(Phase.FALLBACK_TRAIN).isEmpty() || !helper.actions.get(Phase.FALLBACK_SIZE_BASED).isEmpty(); } } interface ActionLog { } abstract class TaskAction implements ActionLog { public TaskAttemptID taskID; public TaskAction(TaskAttemptID taskID) { this.taskID = taskID; } } class AssignLog extends TaskAction { public AssignLog(TaskAttemptID attemptID) { super(attemptID); } @Override public String toString() { return "start(" + this.taskID + ")"; } } class ResumeLog extends TaskAction { public ResumeLog(TaskAttemptID attemptID) { super(attemptID); } @Override public String toString() { return "resume(" + this.taskID + ")"; } } class SuspendLog extends TaskAction { public JobID jobID; public SuspendLog(TaskAttemptID suspended, JobID reasonJob) { super(suspended); this.jobID = reasonJob; } public String getReason() { return this.jobID.toString(); } @Override public String toString() { return "suspend(" + this.taskID + ", reason: " + this.getReason() + ")"; } } class SuspendForTrainLog extends SuspendLog { public SuspendForTrainLog(TaskAttemptID suspended, JobID reasonJob) { super(suspended, reasonJob); } @Override public String getReason() { return "train of " + this.jobID; } } class SuspendBiggerLog extends SuspendLog { private long sizeSuspended; private long sizeReason; public SuspendBiggerLog(TaskAttemptID suspended, JobID reasonJob, long l, long m) { super(suspended, reasonJob); this.sizeSuspended = l; this.sizeReason = m; } @Override public String getReason() { return this.jobID + " " + sizeReason + " < " + sizeSuspended; } } class KillLog extends TaskAction { public JobID jobID; public KillLog(TaskAttemptID suspended, JobID reasonJob) { super(suspended); if (suspended.getJobID().equals(reasonJob)) throw new RuntimeException(suspended.getJobID() + " " + reasonJob); this.jobID = reasonJob; } public String getReason() { return this.jobID.toString(); } @Override public String toString() { return "kill(" + this.taskID + ", reason: " + this.getReason() + ")"; } } class KillForTrainLog extends KillLog { public KillForTrainLog(TaskAttemptID suspended, JobID reasonJob) { super(suspended, reasonJob); } @Override public String getReason() { return "train of " + this.jobID; } } class KillBiggerLog extends KillLog { private long sizeSuspended; private long sizeReason; public KillBiggerLog(TaskAttemptID suspended, JobID reasonJob, long l, long m) { super(suspended, reasonJob); this.sizeSuspended = l; this.sizeReason = m; } @Override public String getReason() { return this.jobID + " " + sizeReason + " < " + sizeSuspended; } }