private void mayBeConstructFinalFullCounters() { // Calculating full-counters. This should happen only once for the job. synchronized (this.fullCountersLock) { if (this.fullCounters != null) { // Already constructed. Just return. return; } this.constructFinalFullcounters(); } }
@Override public void transition(JobImpl job, JobEvent event) { //succeeded map task is restarted back job.decrementSucceededMapperCount(); } }
@Override public JobReport getReport() { readLock.lock(); try { JobState state = getState(); // jobFile can be null if the job is not yet inited. String jobFile = remoteJobConfFile == null ? "" : remoteJobConfFile.toString(); StringBuilder diagsb = new StringBuilder(); for (String s : getDiagnostics()) { diagsb.append(s).append("\n"); } if (getInternalState() == JobStateInternal.NEW) { return MRBuilderUtils.newJobReport(jobId, jobName, username, state, appSubmitTime, startTime, finishTime, setupProgress, 0.0f, 0.0f, cleanupProgress, jobFile, amInfos, isUber, diagsb.toString()); } computeProgress(); JobReport report = MRBuilderUtils.newJobReport(jobId, jobName, username, state, appSubmitTime, startTime, finishTime, setupProgress, this.mapProgress, this.reduceProgress, cleanupProgress, jobFile, amInfos, isUber, diagsb.toString()); return report; } finally { readLock.unlock(); } }
public void run() { LOG.info("Sending event " + toSend + " to " + job.getID()); job.getEventHandler().handle(toSend); } }
protected JobStateInternal checkReadyForCommit() { JobStateInternal currentState = getInternalState(); if (completedTaskCount == tasks.size() && currentState == JobStateInternal.RUNNING) { eventHandler.handle(new CommitterJobCommitEvent(jobId, getJobContext())); return JobStateInternal.COMMITTING; } // return the current state as job not ready to commit yet return getInternalState(); }
@Override public JobState getState() { readLock.lock(); try { JobState state = getExternalState(getInternalState()); if (!appContext.hasSuccessfullyUnregistered() && (state == JobState.SUCCEEDED || state == JobState.FAILED || state == JobState.KILLED || state == JobState.ERROR)) { return lastNonFinalState; } else { return state; } } finally { readLock.unlock(); } }
JobStateInternal oldState = getInternalState(); try { getStateMachine().doTransition(event.getType(), event); } catch (InvalidStateTransitonException e) { LOG.error("Can't handle this event at current state", e); addDiagnostic("Invalid event " + event.getType() + " on Job " + this.jobId); eventHandler.handle(new JobEvent(this.jobId, if (oldState != getInternalState()) { LOG.info(jobId + "Job Transitioned from " + oldState + " to " + getInternalState()); rememberLastNonFinalState(oldState);
job.failedReduceTaskCount*100 > job.allowedReduceFailuresPercent*job.numReduceTasks) { job.setFinishTime(); " failedReduces:" + job.failedReduceTaskCount; LOG.info(diagnosticMsg); job.addDiagnostic(diagnosticMsg); new TriggerScheduledFuture(job, new JobEvent(job.getID(), JobEventType.JOB_FAIL_WAIT_TIMEDOUT)), job.conf.getInt( MRJobConfig.MR_AM_COMMITTER_CANCEL_TIMEOUT_MS, return job.checkReadyForCommit();
@Override public Counters getAllCounters() { readLock.lock(); try { JobStateInternal state = getInternalState(); if (state == JobStateInternal.ERROR || state == JobStateInternal.FAILED || state == JobStateInternal.KILLED || state == JobStateInternal.SUCCEEDED) { this.mayBeConstructFinalFullCounters(); return fullCounters; } Counters counters = new Counters(); counters.incrAllCounters(jobCounters); return incrTaskCounters(counters, tasks.values()); } finally { readLock.unlock(); } }
when(mockContext.hasSuccessfullyUnregistered()).thenReturn(false); JobImpl job = createStubbedJob(conf, dispatcher, 2, mockContext); JobId jobId = job.getID(); job.handle(new JobEvent(jobId, JobEventType.JOB_INIT)); assertJobState(job, JobStateInternal.INITED); job.handle(new JobStartEvent(jobId)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_COMPLETED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_COMPLETED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_MAP_TASK_RESCHEDULED)); assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE)); assertJobState(job, JobStateInternal.FAILED); Assert.assertEquals(JobState.RUNNING, job.getState()); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true); Assert.assertEquals(JobState.FAILED, job.getState());
@Override protected JobStateInternal checkJobAfterTaskCompletion(JobImpl job) { if (job.completedTaskCount == job.tasks.size()) { job.setFinishTime(); job.eventHandler.handle(new CommitterJobAbortEvent(job.jobId, job.jobContext, org.apache.hadoop.mapreduce.JobStatus.State.KILLED)); return JobStateInternal.KILL_ABORT; } //return the current state, Job not finished yet return job.getInternalState(); } }
private static void completeJobTasks(JobImpl job) { // complete the map tasks and the reduce tasks so we start committing int numMaps = job.getTotalMaps(); for (int i = 0; i < numMaps; ++i) { job.handle(new JobTaskEvent( MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.SUCCEEDED)); Assert.assertEquals(JobState.RUNNING, job.getState()); } int numReduces = job.getTotalReduces(); for (int i = 0; i < numReduces; ++i) { job.handle(new JobTaskEvent( MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.SUCCEEDED)); Assert.assertEquals(JobState.RUNNING, job.getState()); } }
/** * This transition executes in the event-dispatcher thread, though it's * triggered in MRAppMaster's startJobs() method. */ @Override public void transition(JobImpl job, JobEvent event) { JobStartEvent jse = (JobStartEvent) event; if (jse.getRecoveredJobStartTime() != 0) { job.startTime = jse.getRecoveredJobStartTime(); } else { job.startTime = job.clock.getTime(); } JobInitedEvent jie = new JobInitedEvent(job.oldJobId, job.startTime, job.numMapTasks, job.numReduceTasks, job.getState().toString(), job.isUber()); job.eventHandler.handle(new JobHistoryEvent(job.jobId, jie)); JobInfoChangeEvent jice = new JobInfoChangeEvent(job.oldJobId, job.appSubmitTime, job.startTime); job.eventHandler.handle(new JobHistoryEvent(job.jobId, jice)); job.metrics.runningJob(job); job.eventHandler.handle(new CommitterJobSetupEvent( job.jobId, job.jobContext)); } }
@Override public void transition(JobImpl job, JobEvent event) { job.addDiagnostic(((JobDiagnosticsUpdateEvent) event) .getDiagnosticUpdate()); } }
assertJobState(job, JobStateInternal.COMMITTING); job.handle(new JobEvent(job.getID(), JobEventType.JOB_TASK_ATTEMPT_COMPLETED)); assertJobState(job, JobStateInternal.COMMITTING); job.handle(new JobEvent(job.getID(), JobEventType.JOB_MAP_TASK_RESCHEDULED)); assertJobState(job, JobStateInternal.COMMITTING); assertJobState(job, JobStateInternal.SUCCEEDED); job.handle(new JobEvent(job.getID(), JobEventType.JOB_TASK_ATTEMPT_COMPLETED)); assertJobState(job, JobStateInternal.SUCCEEDED); job.handle(new JobEvent(job.getID(), JobEventType.JOB_MAP_TASK_RESCHEDULED)); assertJobState(job, JobStateInternal.SUCCEEDED);
private static void assertJobState(JobImpl job, JobStateInternal state) { int timeToWaitMsec = 5 * 1000; while (timeToWaitMsec > 0 && job.getInternalState() != state) { try { Thread.sleep(10); timeToWaitMsec -= 10; } catch (InterruptedException e) { break; } } Assert.assertEquals(state, job.getInternalState()); }
@Override protected Job createJob(Configuration conf, JobStateInternal forcedState, String diagnostic) { JobImpl jobImpl = mock(JobImpl.class); when(jobImpl.getInternalState()).thenReturn(this.jobStateInternal); when(jobImpl.getAllCounters()).thenReturn(new Counters()); JobID jobID = JobID.forName("job_1234567890000_0001"); JobId jobId = TypeConverter.toYarn(jobID); when(jobImpl.getID()).thenReturn(jobId); ((AppContext) getContext()) .getAllJobs().put(jobImpl.getID(), jobImpl); return jobImpl; }
private void actOnUnusableNode(NodeId nodeId, NodeState nodeState) { // rerun previously successful map tasks // do this only if the job is still in the running state and there are // running reducers if (getInternalState() == JobStateInternal.RUNNING && !allReducersComplete()) { List<TaskAttemptId> taskAttemptIdList = nodesToSucceededTaskAttempts.get(nodeId); if (taskAttemptIdList != null) { String mesg = "TaskAttempt killed because it ran on unusable node " + nodeId; for (TaskAttemptId id : taskAttemptIdList) { if (TaskType.MAP == id.getTaskId().getTaskType()) { // reschedule only map tasks because their outputs maybe unusable LOG.info(mesg + ". AttemptId:" + id); // Kill the attempt and indicate that next map attempt should be // rescheduled (i.e. considered as a fast fail map). eventHandler.handle(new TaskAttemptKillEvent(id, mesg, true)); } } } } // currently running task attempts on unusable nodes are handled in // RMContainerAllocator }
job.fs = job.getFileSystem(job.conf); job.addDiagnostic("No of maps and reduces are 0 " + job.jobId); } else if (job.numMapTasks == 0) { job.reduceWeight = 0.9f; LOG.warn("Job init failed", e); job.metrics.endPreparingJob(job); job.addDiagnostic("Job init failed : " + StringUtils.stringifyException(e));
@Test public void testMetaInfoSizeOverMax() throws Exception { Configuration conf = new Configuration(); JobID jobID = JobID.forName("job_1234567890000_0001"); JobId jobId = TypeConverter.toYarn(jobID); MRAppMetrics mrAppMetrics = MRAppMetrics.create(); JobImpl job = new JobImpl(jobId, ApplicationAttemptId.newInstance( ApplicationId.newInstance(0, 0), 0), conf, mock(EventHandler.class), null, new JobTokenSecretManager(), new Credentials(), null, null, mrAppMetrics, null, true, null, 0, null, null, null, null); InitTransition initTransition = new InitTransition() { @Override protected TaskSplitMetaInfo[] createSplits(JobImpl job, JobId jobId) { throw new YarnRuntimeException(EXCEPTIONMSG); } }; JobEvent mockJobEvent = mock(JobEvent.class); JobStateInternal jobSI = initTransition.transition(job, mockJobEvent); Assert.assertTrue("When init fails, return value from InitTransition.transition should equal NEW.", jobSI.equals(JobStateInternal.NEW)); Assert.assertTrue("Job diagnostics should contain YarnRuntimeException", job.getDiagnostics().toString().contains("YarnRuntimeException")); Assert.assertTrue("Job diagnostics should contain " + EXCEPTIONMSG, job.getDiagnostics().toString().contains(EXCEPTIONMSG)); }