/** * @param taskCtx Task context. * @return Output. * @throws IgniteCheckedException If failed. */ public HadoopTaskOutput output(HadoopTaskContext taskCtx) throws IgniteCheckedException { switch (taskCtx.taskInfo().type()) { case MAP: assert !job.info().hasCombiner() : "The output creation is allowed if combiner has not been defined."; case COMBINE: return new PartitionedOutput(taskCtx); default: throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type()); } }
/** {@inheritDoc} */ @Override public void cleanupTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException { HadoopTaskContext ctx = ctxs.remove(new T2<>(info.type(), info.taskNumber())).get(); taskCtxClsPool.add(ctx.getClass()); File locDir = taskLocalDir(igniteWorkDirectory(), locNodeId, info); if (locDir.exists()) U.delete(locDir); }
/** * Gets name for the task class loader. Task class loader * @param info The task info. * @param prefix Get only prefix (without task type and number) * @return The class loader name. */ public static String nameForTask(HadoopTaskInfo info, boolean prefix) { if (prefix) return "hadoop-task-" + info.jobId() + "-"; else return "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber(); }
/** * Generate name that consists of some event information. * * @param info Task info. * @param evtType The type of the event. * @return String contains necessary event information. */ private String eventName(HadoopTaskInfo info, String evtType) { return eventName(info.type().toString(), info.taskNumber(), evtType); }
/** * Notifies node about task finish. * * @param run Finished task runnable. * @param status Task status. */ private void onTaskFinished0(HadoopRunnableTask run, HadoopTaskStatus status) { HadoopTaskInfo info = run.taskInfo(); int pendingTasks0 = pendingTasks.decrementAndGet(); if (log.isDebugEnabled()) log.debug("Hadoop task execution finished [info=" + info + ", state=" + status.state() + ", waitTime=" + run.waitTime() + ", execTime=" + run.executionTime() + ", pendingTasks=" + pendingTasks0 + ", err=" + status.failCause() + ']'); assert info.type() == MAP || info.type() == REDUCE : "Only MAP or REDUCE tasks are supported."; boolean flush = pendingTasks0 == 0 && info.type() == MAP; notifyTaskFinished(info, status, flush); }
/** * Returns subdirectory of job working directory for task execution. * * @param workDir Work directory. * @param locNodeId Local node ID. * @param info Task info. * @return Working directory for task. * @throws IgniteCheckedException If Failed. */ public static File taskLocalDir(String workDir, UUID locNodeId, HadoopTaskInfo info) throws IgniteCheckedException { File jobLocDir = jobLocalDir(workDir, locNodeId, info.jobId()); return new File(jobLocDir, info.type() + "_" + info.taskNumber() + "_" + info.attempt()); }
/** * Adds event of the task finish. * * @param info Task info. * @param ts Timestamp of the event. */ public void onTaskFinish(HadoopTaskInfo info, long ts) { if (info.type() == HadoopTaskType.REDUCE && lastShuffleMsg != null) { evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "start"), firstShuffleMsg)); evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "finish"), lastShuffleMsg)); lastShuffleMsg = null; } evts.add(new T2<>(eventName(info, "finish"), ts)); }
/** * @param taskCtx Task context. * @return Input. * @throws IgniteCheckedException If failed. */ public HadoopTaskInput input(HadoopTaskContext taskCtx) throws IgniteCheckedException { switch (taskCtx.taskInfo().type()) { case REDUCE: int reducer = taskCtx.taskInfo().taskNumber(); HadoopMultimap m = locMaps.get(reducer); if (m != null) return m.input(taskCtx); return new HadoopTaskInput() { // Empty input. @Override public boolean next() { return false; } @Override public Object key() { throw new IllegalStateException(); } @Override public Iterator<?> values() { throw new IllegalStateException(); } @Override public void close() { // No-op. } }; default: throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type()); } }
/** * @param ctx Task context. * @return Task input. * @throws IgniteCheckedException If failed. */ private HadoopTaskInput createInputInternal(HadoopTaskContext ctx) throws IgniteCheckedException { switch (ctx.taskInfo().type()) { case SETUP: case MAP: case COMMIT: case ABORT: return null; case COMBINE: assert combinerInput != null; return combinerInput.input(ctx); default: return createInput(ctx); } }
T2<HadoopTaskType, Integer> locTaskId = new T2<>(info.type(), info.taskNumber());
/** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public Comparator<Object> groupComparator() { Comparator<?> res; switch (taskInfo().type()) { case COMBINE: res = COMBINE_KEY_GROUPING_SUPPORTED ? jobContext().getCombinerKeyGroupingComparator() : jobContext().getGroupingComparator(); break; case REDUCE: res = jobContext().getGroupingComparator(); break; default: return null; } if (res != null && res.getClass() != sortComparator().getClass()) return (Comparator<Object>)res; return null; }
/** * @param task Task. */ private void startThread(final Callable<?> task) { String workerName; if (task instanceof HadoopRunnableTask) { final HadoopTaskInfo i = ((HadoopRunnableTask)task).taskInfo(); workerName = "Hadoop-task-" + i.jobId() + "-" + i.type() + "-" + i.taskNumber() + "-" + i.attempt(); } else workerName = task.toString(); GridWorker w = new GridWorker(igniteInstanceName, workerName, log, lsnr) { @Override protected void body() { try { task.call(); } catch (Exception e) { log.error("Failed to execute task: " + task, e); } } }; workers.add(w); if (shutdown) w.cancel(); new IgniteThread(w).start(); }
"Invalid task status [info=" + info + ", status=" + status + ']'; assert state != null || (ctx.jobUpdateLeader() && (info.type() == COMMIT || info.type() == ABORT)): "Missing local state for finished task [info=" + info + ", status=" + status + ']'; incrCntrs = new IncrementCountersProcessor(null, status.counters()); switch (info.type()) { case SETUP: { state.onSetupFinished(info, status, incrCntrs);
/** * Creates Hadoop attempt ID. * * @return Attempt ID. */ public TaskAttemptID attemptId() { TaskID tid = new TaskID(jobCtx.getJobID(), taskType(taskInfo().type()), taskInfo().taskNumber()); return new TaskAttemptID(tid, taskInfo().attempt()); }
/** {@inheritDoc} */ @Override public void prepareTaskEnvironment() throws IgniteCheckedException { File locDir; switch(taskInfo().type()) { case MAP: case REDUCE: job().prepareTaskEnvironment(taskInfo()); locDir = taskLocalDir(job.igniteWorkDirectory(), locNodeId, taskInfo()); break; default: locDir = jobLocalDir(job.igniteWorkDirectory(), locNodeId, taskInfo().jobId()); } ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf().getClassLoader()); try { FileSystem.get(jobConf()); LocalFileSystem locFs = FileSystem.getLocal(jobConf()); locFs.setWorkingDirectory(new Path(locDir.getAbsolutePath())); } catch (Throwable e) { if (e instanceof Error) throw (Error)e; throw transformException(e); } finally { HadoopCommonUtils.restoreContextClassLoader(oldLdr); } }
HadoopProcess proc = runningProcsByJobId.get(job.id()); HadoopTaskType taskType = F.first(tasks).type();
/** * Creates appropriate task from current task info. * * @return Task. */ private HadoopTask createTask() { boolean isAbort = taskInfo().type() == HadoopTaskType.ABORT; switch (taskInfo().type()) { case SETUP: return useNewMapper ? new HadoopV2SetupTask(taskInfo()) : new HadoopV1SetupTask(taskInfo()); case MAP: return useNewMapper ? new HadoopV2MapTask(taskInfo()) : new HadoopV1MapTask(taskInfo()); case REDUCE: return useNewReducer ? new HadoopV2ReduceTask(taskInfo(), true) : new HadoopV1ReduceTask(taskInfo(), true); case COMBINE: return useNewCombiner ? new HadoopV2ReduceTask(taskInfo(), false) : new HadoopV1ReduceTask(taskInfo(), false); case COMMIT: case ABORT: return useNewReducer ? new HadoopV2CleanupTask(taskInfo(), isAbort) : new HadoopV1CleanupTask(taskInfo(), isAbort); default: return null; } }
/** * @param ctx Task info. * @return Task output. * @throws IgniteCheckedException If failed. */ private HadoopTaskOutput createOutputInternal(HadoopTaskContext ctx) throws IgniteCheckedException { switch (ctx.taskInfo().type()) { case SETUP: case REDUCE: case COMMIT: case ABORT: return null; case MAP: if (job.info().hasCombiner()) { assert combinerInput == null; combinerInput = get(job.info(), SHUFFLE_COMBINER_NO_SORTING, false) ? new HadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)): new HadoopSkipList(job.info(), mem); // TODO replace with red-black tree return combinerInput.startAdding(ctx); } default: return createOutput(ctx); } }
if (info.type() == MAP && job.info().hasCombiner()) {