/** * Submit the work for actual execution. * * @throws InvalidProtocolBufferException */ public void submitWork(SubmitWorkRequestProto request, String llapHost, int llapPort) { // Register the pending events to be sent for this spec. VertexOrBinary vob = request.getWorkSpec(); assert vob.hasVertexBinary() != vob.hasVertex(); SignableVertexSpec vertex = null; try { vertex = vob.hasVertex() ? vob.getVertex() : SignableVertexSpec.parseFrom(vob.getVertexBinary()); } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } QueryIdentifierProto queryIdentifierProto = vertex.getQueryIdentifier(); TezTaskAttemptID attemptId = Converters.createTaskAttemptId( queryIdentifierProto, vertex.getVertexIndex(), request.getFragmentNumber(), request.getAttemptNumber()); final String fragmentId = attemptId.toString(); pendingEvents.putIfAbsent( fragmentId, new PendingEventData( new TaskHeartbeatInfo(fragmentId, llapHost, llapPort), Lists.<TezEvent>newArrayList())); // Setup timer task to check for hearbeat timeouts timer.scheduleAtFixedRate( new HeartbeatCheckTask(), connectionTimeout, connectionTimeout, TimeUnit.MILLISECONDS); // Send out the actual SubmitWorkRequest communicator.sendSubmitWork( request, llapHost, llapPort, new LlapProtocolClientProxy.ExecuteRequestCallback<SubmitWorkResponseProto>() { @Override public void setResponse(SubmitWorkResponseProto response) { if (response.hasSubmissionState()) { if (response.getSubmissionState().equals(SubmissionStateProto.REJECTED)) { String msg = "Fragment: " + fragmentId + " rejected. Server Busy."; LOG.info(msg); if (responder != null) { Throwable err = new RuntimeException(msg); responder.submissionFailed(fragmentId, err); } return; } } } @Override public void indicateError(Throwable t) { String msg = "Failed to submit: " + fragmentId; LOG.error(msg, t); Throwable err = new RuntimeException(msg, t); responder.submissionFailed(fragmentId, err); } }); }
@SuppressWarnings("unchecked") public void sendTezEventToSourceTasks(TezEvent tezEvent) { if (!bufferEvents.get()) { switch (tezEvent.getEventType()) { case INPUT_READ_ERROR_EVENT: InputReadErrorEvent event = (InputReadErrorEvent) tezEvent.getEvent(); TezTaskAttemptID destAttemptId = tezEvent.getSourceInfo().getTaskAttemptID(); int destTaskIndex = destAttemptId.getTaskID().getId(); int srcTaskIndex = edgeManager.routeEventToSourceTasks(destTaskIndex, event); int numConsumers = edgeManager.getDestinationConsumerTaskNumber( srcTaskIndex, destinationVertex.getTotalTasks()); TezTaskID srcTaskId = sourceVertex.getTask(srcTaskIndex).getTaskId(); int taskAttemptIndex = event.getVersion(); TezTaskAttemptID srcTaskAttemptId = new TezTaskAttemptID(srcTaskId, taskAttemptIndex); eventHandler.handle( new TaskAttemptEventOutputFailed(srcTaskAttemptId, tezEvent, numConsumers)); break; default: throw new TezUncheckedException("Unhandled tez event type: " + tezEvent.getEventType()); } } else { sourceEventBuffer.add(tezEvent); } }
@Test public void testLocalResourceAddition() { WrappedContainer wc = new WrappedContainer(); String rsrc1 = "rsrc1"; String rsrc2 = "rsrc2"; String rsrc3 = "rsrc3"; Map<String, LocalResource> initialResources = Maps.newHashMap(); initialResources.put(rsrc1, createLocalResource(rsrc1)); wc.launchContainer(initialResources, new Credentials()); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); AMContainerTask task1 = wc.pullTaskToRun(); assertEquals(0, task1.getAdditionalResources().size()); wc.taskAttemptSucceeded(wc.taskAttemptID); // Add some resources to the next task. Map<String, LocalResource> additionalResources = Maps.newHashMap(); additionalResources.put(rsrc2, createLocalResource(rsrc2)); additionalResources.put(rsrc3, createLocalResource(rsrc3)); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2, additionalResources, new Credentials()); AMContainerTask task2 = wc.pullTaskToRun(); Map<String, LocalResource> pullTaskAdditionalResources = task2.getAdditionalResources(); assertEquals(2, pullTaskAdditionalResources.size()); pullTaskAdditionalResources.remove(rsrc2); pullTaskAdditionalResources.remove(rsrc3); assertEquals(0, pullTaskAdditionalResources.size()); wc.taskAttemptSucceeded(taID2); // Verify Resources registered for this container. Map<String, LocalResource> containerLRs = new HashMap<String, LocalResource>(wc.amContainer.containerLocalResources); assertEquals(3, containerLRs.size()); containerLRs.remove(rsrc1); containerLRs.remove(rsrc2); containerLRs.remove(rsrc3); assertEquals(0, containerLRs.size()); // Try launching another task with the same reosurces as Task2. Verify the // task is not asked to re-localize again. TezTaskAttemptID taID3 = TezTaskAttemptID.getInstance(wc.taskID, 3); wc.assignTaskAttempt(taID3, new HashMap<String, LocalResource>(), new Credentials()); AMContainerTask task3 = wc.pullTaskToRun(); assertEquals(0, task3.getAdditionalResources().size()); wc.taskAttemptSucceeded(taID3); // Verify references are cleared after a container completes. wc.containerCompleted(false); assertNull(wc.amContainer.containerLocalResources); assertNull(wc.amContainer.additionalLocalResources); }
public void fromProto(TaskAttemptStartedProto proto) { this.taskAttemptId = TezTaskAttemptID.fromString(proto.getTaskAttemptId()); this.launchTime = proto.getStartTime(); this.containerId = ConverterUtils.toContainerId(proto.getContainerId()); this.nodeId = ConverterUtils.toNodeId(proto.getNodeId()); this.creationTime = proto.getCreationTime(); this.allocationTime = proto.getAllocationTime(); if (proto.hasCreationCausalTA()) { this.creationCausalTA = TezTaskAttemptID.fromString(proto.getCreationCausalTA()); } }
@Test(timeout = 10000) public void testBasicSpeculationPerVertexConf() throws Exception { DAG dag = DAG.create("test"); String vNameNoSpec = "A"; String vNameSpec = "B"; Vertex vA = Vertex.create(vNameNoSpec, ProcessorDescriptor.create("Proc.class"), 5); Vertex vB = Vertex.create(vNameSpec, ProcessorDescriptor.create("Proc.class"), 5); vA.setConf(TezConfiguration.TEZ_AM_SPECULATION_ENABLED, "false"); dag.addVertex(vA); dag.addVertex(vB); // min/max src fraction is set to 1. So vertices will run sequentially dag.addEdge( Edge.create( vA, vB, EdgeProperty.create( DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, OutputDescriptor.create("O"), InputDescriptor.create("I")))); MockTezClient tezClient = createTezSession(); DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = dagImpl.getVertex(vNameSpec).getVertexId(); TezVertexID vertexIdNoSpec = dagImpl.getVertex(vNameNoSpec).getVertexId(); // original attempt is killed and speculative one is successful TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TezTaskAttemptID noSpecTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexIdNoSpec, 0), 0); // cause speculation trigger for both mockLauncher.setStatusUpdatesForTask(killedTaId, 100); mockLauncher.setStatusUpdatesForTask(noSpecTaId, 100); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); org.apache.tez.dag.app.dag.Vertex vSpec = dagImpl.getVertex(vertexId); org.apache.tez.dag.app.dag.Vertex vNoSpec = dagImpl.getVertex(vertexIdNoSpec); // speculation for vA but not for vB Assert.assertTrue( vSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue() > 0); Assert.assertEquals( 0, vNoSpec.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue()); tezClient.stop(); }
public TaskAttemptStartedProto toProto() { TaskAttemptStartedProto.Builder builder = TaskAttemptStartedProto.newBuilder(); builder .setTaskAttemptId(taskAttemptId.toString()) .setStartTime(launchTime) .setContainerId(containerId.toString()) .setNodeId(nodeId.toString()) .setCreationTime(creationTime) .setAllocationTime(allocationTime); if (creationCausalTA != null) { builder.setCreationCausalTA(creationCausalTA.toString()); } return builder.build(); }
public void sendTezEventToDestinationTasks(TezEvent tezEvent) { if (!bufferEvents.get()) { List<Integer> destTaskIndices = new ArrayList<Integer>(); boolean isDataMovementEvent = true; switch (tezEvent.getEventType()) { case INPUT_FAILED_EVENT: isDataMovementEvent = false; case DATA_MOVEMENT_EVENT: Event event = tezEvent.getEvent(); TezTaskAttemptID sourceAttemptId = tezEvent.getSourceInfo().getTaskAttemptID(); int sourceTaskIndex = sourceAttemptId.getTaskID().getId(); if (isDataMovementEvent) { edgeManager.routeEventToDestinationTasks( (DataMovementEvent) event, sourceTaskIndex, destinationVertex.getTotalTasks(), destTaskIndices); } else { edgeManager.routeEventToDestinationTasks( (InputFailedEvent) event, sourceTaskIndex, destinationVertex.getTotalTasks(), destTaskIndices); } for (Integer destTaskIndex : destTaskIndices) { EventMetaData destMeta = new EventMetaData( EventProducerConsumerType.INPUT, destinationVertex.getName(), sourceVertex.getName(), null); // will be filled by Task when sending the event. Is it needed? if (isDataMovementEvent) { destMeta.setIndex(((DataMovementEvent) event).getTargetIndex()); } else { destMeta.setIndex(((InputFailedEvent) event).getTargetIndex()); } tezEvent.setDestinationInfo(destMeta); TezTaskID destTaskId = destinationVertex.getTask(destTaskIndex).getTaskId(); sendEventToTask(destTaskId, tezEvent); } break; default: throw new TezUncheckedException("Unhandled tez event type: " + tezEvent.getEventType()); } } else { destinationEventBuffer.add(tezEvent); } }
@SuppressWarnings("rawtypes") @Test public void testNodeFailedAtCompletedMultipleSuccessfulTAs() { WrappedContainer wc = new WrappedContainer(); List<Event> outgoingEvents; wc.launchContainer(); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); wc.pullTaskToRun(); wc.taskAttemptSucceeded(wc.taskAttemptID); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2); wc.pullTaskToRun(); wc.taskAttemptSucceeded(taID2); wc.stopRequest(); wc.nmStopSent(); wc.containerCompleted(false); wc.verifyState(AMContainerState.COMPLETED); wc.nodeFailed(); outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2); verifyUnOrderedOutgoingEventTypes( outgoingEvents, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_NODE_FAILED); assertNull(wc.amContainer.getRunningTaskAttempt()); assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size()); assertEquals(2, wc.amContainer.getAllTaskAttempts().size()); }
@Test public void testDuplicateCompletedEvents() { WrappedContainer wc = new WrappedContainer(); wc.launchContainer(); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); wc.pullTaskToRun(); wc.taskAttemptSucceeded(wc.taskAttemptID); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2); wc.pullTaskToRun(); wc.taskAttemptSucceeded(taID2); wc.stopRequest(); wc.nmStopSent(); wc.containerCompleted(false); wc.verifyState(AMContainerState.COMPLETED); wc.verifyNoOutgoingEvents(); wc.containerCompleted(false); wc.verifyNoOutgoingEvents(); wc.verifyHistoryStopEvent(); }
@SuppressWarnings("rawtypes") @Test public void testTaskAssignedToCompletedContainer() { WrappedContainer wc = new WrappedContainer(); List<Event> outgoingEvents; wc.launchContainer(); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); wc.pullTaskToRun(); wc.taskAttemptSucceeded(wc.taskAttemptID); wc.containerCompleted(false); wc.verifyState(AMContainerState.COMPLETED); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2); outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1); verifyUnOrderedOutgoingEventTypes(outgoingEvents, TaskAttemptEventType.TA_CONTAINER_TERMINATED); TaskAttemptEventContainerTerminated ctEvent = (TaskAttemptEventContainerTerminated) outgoingEvents.get(0); assertEquals(taID2, ctEvent.getTaskAttemptID()); wc.verifyHistoryStopEvent(); // Allocation to a completed Container is considered an error. // TODO Is this valid ? assertTrue(wc.amContainer.isInErrorState()); }
private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) { if (commitAttempt != null && commitAttempt.equals(attempt)) { LOG.info("Removing commit attempt: " + commitAttempt); commitAttempt = null; } if (attempt != null && !attempt.isFinished()) { eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(), logMsg)); } }
public void testBasicSpeculation(boolean withProgress) throws Exception { DAG dag = DAG.create("test"); Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5); dag.addVertex(vA); MockTezClient tezClient = createTezSession(); DAGClient dagClient = tezClient.submitDAG(dag); DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG(); TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), 0); // original attempt is killed and speculative one is successful TezTaskAttemptID killedTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0); TezTaskAttemptID successTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 1); mockLauncher.updateProgress(withProgress); // cause speculation trigger mockLauncher.setStatusUpdatesForTask(killedTaId, 100); mockLauncher.startScheduling(true); dagClient.waitForCompletion(); Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(2, task.getAttempts().size()); Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals( TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, killedAttempt.getTerminationCause()); if (withProgress) { // without progress updates occasionally more than 1 task speculates Assert.assertEquals( 1, task.getCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue()); Assert.assertEquals( 1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue()); org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); Assert.assertEquals( 1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS).getValue()); } tezClient.stop(); }
@Test(timeout = 5000) public void testDagNumber() { String[] localDirs = new String[] {"dummyLocalDir"}; int appAttemptNumber = 1; TezUmbilical tezUmbilical = mock(TezUmbilical.class); String dagName = "DAG_NAME"; String vertexName = "VERTEX_NAME"; int vertexParallelism = 20; int dagNumber = 52; ApplicationId appId = ApplicationId.newInstance(10000, 13); TezDAGID dagId = TezDAGID.getInstance(appId, dagNumber); TezVertexID vertexId = TezVertexID.getInstance(dagId, 6); TezTaskID taskId = TezTaskID.getInstance(vertexId, 4); TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 2); LogicalIOProcessorRuntimeTask runtimeTask = mock(LogicalIOProcessorRuntimeTask.class); doReturn(new TezCounters()).when(runtimeTask).addAndGetTezCounter(any(String.class)); Map<String, ByteBuffer> serviceConsumerMetadata = Maps.newHashMap(); Map<String, String> auxServiceEnv = Maps.newHashMap(); MemoryDistributor memDist = mock(MemoryDistributor.class); ProcessorDescriptor processorDesc = mock(ProcessorDescriptor.class); InputReadyTracker inputReadyTracker = mock(InputReadyTracker.class); ObjectRegistry objectRegistry = new ObjectRegistryImpl(); ExecutionContext execContext = new ExecutionContextImpl("localhost"); long memAvailable = 10000l; TezProcessorContextImpl procContext = new TezProcessorContextImpl( new Configuration(), localDirs, appAttemptNumber, tezUmbilical, dagName, vertexName, vertexParallelism, taskAttemptId, null, runtimeTask, serviceConsumerMetadata, auxServiceEnv, memDist, processorDesc, inputReadyTracker, objectRegistry, execContext, memAvailable); assertEquals(dagNumber, procContext.getDagIdentifier()); assertEquals(appAttemptNumber, procContext.getDAGAttemptNumber()); assertEquals(appId, procContext.getApplicationId()); assertEquals(dagName, procContext.getDAGName()); assertEquals(vertexName, procContext.getTaskVertexName()); assertEquals(vertexId.getId(), procContext.getTaskVertexIndex()); assertTrue(Arrays.equals(localDirs, procContext.getWorkDirs())); }
@Before public void setup() { applicationId = ApplicationId.newInstance(9999l, 1); applicationAttemptId = ApplicationAttemptId.newInstance(applicationId, 1); tezDAGID = TezDAGID.getInstance(applicationId, random.nextInt()); tezVertexID = TezVertexID.getInstance(tezDAGID, random.nextInt()); tezTaskID = TezTaskID.getInstance(tezVertexID, random.nextInt()); tezTaskAttemptID = TezTaskAttemptID.getInstance(tezTaskID, random.nextInt()); dagPlan = DAGPlan.newBuilder().setName("DAGPlanMock").build(); containerId = ContainerId.newInstance(applicationAttemptId, 111); nodeId = NodeId.newInstance("node", 13435); }
@Override public void taskKilled(TezTaskAttemptID taskAttemptId) throws IOException { String taskAttemptIdString = taskAttemptId.toString(); LOG.error("Task killed - " + taskAttemptIdString); registeredTasks.remove(taskAttemptIdString); try { if (responder != null) { responder.taskKilled(taskAttemptId); } } catch (Exception err) { LOG.error("Error during responder execution", err); } }
public TaskFinishedProto toProto() { TaskFinishedProto.Builder builder = TaskFinishedProto.newBuilder(); builder.setTaskId(taskID.toString()).setState(state.ordinal()).setFinishTime(finishTime); if (diagnostics != null) { builder.setDiagnostics(diagnostics); } if (tezCounters != null) { builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters)); } if (successfulAttemptID != null) { builder.setSuccessfulTaskAttemptId(successfulAttemptID.toString()); } return builder.build(); }
public void fromProto(TaskFinishedProto proto) { this.taskID = TezTaskID.fromString(proto.getTaskId()); this.finishTime = proto.getFinishTime(); this.state = TaskState.values()[proto.getState()]; if (proto.hasDiagnostics()) { this.diagnostics = proto.getDiagnostics(); } if (proto.hasCounters()) { this.tezCounters = DagTypeConverters.convertTezCountersFromProto(proto.getCounters()); } if (proto.hasSuccessfulTaskAttemptId()) { this.successfulAttemptID = TezTaskAttemptID.fromString(proto.getSuccessfulTaskAttemptId()); } }
@Override public boolean canCommit(TezTaskAttemptID taskAttemptID) { writeLock.lock(); try { if (getState() != TaskState.RUNNING) { LOG.info("Task not running. Issuing kill to bad commit attempt " + taskAttemptID); eventHandler.handle( new TaskAttemptEventKillRequest(taskAttemptID, "Task not running. Bad attempt.")); return false; } if (commitAttempt == null) { TaskAttempt ta = getAttempt(taskAttemptID); if (ta == null) { throw new TezUncheckedException("Unknown task for commit: " + taskAttemptID); } // Its ok to get a non-locked state snapshot since we handle changes of // state in the task attempt. Dont want to deadlock here. TaskAttemptState taState = ta.getStateNoLock(); if (taState == TaskAttemptState.RUNNING) { commitAttempt = taskAttemptID; LOG.info(taskAttemptID + " given a go for committing the task output."); return true; } else { LOG.info( taskAttemptID + " with state: " + taState + " given a no-go for commit because its not running."); return false; } } else { if (commitAttempt.equals(taskAttemptID)) { LOG.info(taskAttemptID + " given a go for committing the task output."); return true; } // Don't think this can be a pluggable decision, so simply raise an // event for the TaskAttempt to delete its output. // Wait for commit attempt to succeed. Dont kill this. If commit // attempt fails then choose a different committer. When commit attempt // succeeds then this and others will be killed LOG.info(commitAttempt + " is current committer. Commit waiting for: " + taskAttemptID); return false; } } finally { writeLock.unlock(); } }
@SuppressWarnings("rawtypes") @Test public void testNodeFailedAtIdleMultipleAttempts() { WrappedContainer wc = new WrappedContainer(); List<Event> outgoingEvents; wc.launchContainer(); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); wc.pullTaskToRun(); wc.taskAttemptSucceeded(wc.taskAttemptID); wc.verifyState(AMContainerState.IDLE); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2); wc.pullTaskToRun(); wc.taskAttemptSucceeded(taID2); wc.verifyState(AMContainerState.IDLE); wc.nodeFailed(); // Expecting a complete event from the RM wc.verifyState(AMContainerState.STOPPING); outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3); verifyUnOrderedOutgoingEventTypes( outgoingEvents, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_NODE_FAILED, AMSchedulerEventType.S_CONTAINER_DEALLOCATE); for (Event event : outgoingEvents) { if (event.getType() == TaskAttemptEventType.TA_NODE_FAILED) { TaskAttemptEventNodeFailed nfEvent = (TaskAttemptEventNodeFailed) event; assertEquals("nodeFailed", nfEvent.getDiagnosticInfo()); } } assertFalse(wc.amContainer.isInErrorState()); wc.containerCompleted(false); wc.verifyNoOutgoingEvents(); wc.verifyHistoryStopEvent(); assertNull(wc.amContainer.getRunningTaskAttempt()); assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size()); assertEquals(2, wc.amContainer.getAllTaskAttempts().size()); }
public WrappedContainer(boolean shouldProfile, String profileString) { applicationID = ApplicationId.newInstance(rmIdentifier, 1); appAttemptID = ApplicationAttemptId.newInstance(applicationID, 1); containerID = ContainerId.newInstance(appAttemptID, 1); nodeID = NodeId.newInstance("host", 12500); nodeHttpAddress = "host:12501"; resource = Resource.newInstance(1024, 1); priority = Priority.newInstance(1); container = Container.newInstance(containerID, nodeID, nodeHttpAddress, resource, priority, null); chh = mock(ContainerHeartbeatHandler.class); InetSocketAddress addr = new InetSocketAddress("localhost", 0); tal = mock(TaskAttemptListener.class); doReturn(addr).when(tal).getAddress(); dagID = TezDAGID.getInstance(applicationID, 1); vertexID = TezVertexID.getInstance(dagID, 1); taskID = TezTaskID.getInstance(vertexID, 1); taskAttemptID = TezTaskAttemptID.getInstance(taskID, 1); eventHandler = mock(EventHandler.class); historyEventHandler = mock(HistoryEventHandler.class); Configuration conf = new Configuration(false); appContext = mock(AppContext.class); doReturn(new HashMap<ApplicationAccessType, String>()).when(appContext).getApplicationACLs(); doReturn(eventHandler).when(appContext).getEventHandler(); doReturn(appAttemptID).when(appContext).getApplicationAttemptId(); doReturn(applicationID).when(appContext).getApplicationID(); doReturn(new SystemClock()).when(appContext).getClock(); doReturn(historyEventHandler).when(appContext).getHistoryHandler(); doReturn(conf).when(appContext).getAMConf(); mockDAGID(); taskSpec = mock(TaskSpec.class); doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID(); amContainer = new AMContainerImpl(container, chh, tal, new ContainerContextMatcher(), appContext); }
@SuppressWarnings("rawtypes") @Test public void testAllocationAtRunning() { WrappedContainer wc = new WrappedContainer(); List<Event> outgoingEvents; wc.launchContainer(); wc.containerLaunched(); wc.assignTaskAttempt(wc.taskAttemptID); wc.pullTaskToRun(); wc.verifyState(AMContainerState.RUNNING); TezTaskAttemptID taID2 = TezTaskAttemptID.getInstance(wc.taskID, 2); wc.assignTaskAttempt(taID2); wc.verifyState(AMContainerState.STOP_REQUESTED); verify(wc.tal).unregisterRunningContainer(wc.containerID); verify(wc.chh).unregister(wc.containerID); // 1 for NM stop request. 2 TERMINATING to TaskAttempt. outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3); verifyUnOrderedOutgoingEventTypes( outgoingEvents, NMCommunicatorEventType.CONTAINER_STOP_REQUEST, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATING); assertTrue(wc.amContainer.isInErrorState()); wc.nmStopSent(); wc.containerCompleted(false); wc.verifyHistoryStopEvent(); // 1 Inform scheduler. 2 TERMINATED to TaskAttempt. outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2); verifyUnOrderedOutgoingEventTypes( outgoingEvents, TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_CONTAINER_TERMINATED); assertNull(wc.amContainer.getRunningTaskAttempt()); assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size()); assertEquals(2, wc.amContainer.getAllTaskAttempts().size()); }
@Override public String toString() { return "vertexName=" + vertexName + ", taskId=" + taskID + ", startTime=" + startTime + ", finishTime=" + finishTime + ", timeTaken=" + (finishTime - startTime) + ", status=" + state.name() + ", successfulAttemptID=" + (successfulAttemptID == null ? "null" : successfulAttemptID.toString()) + ", diagnostics=" + diagnostics + ", counters=" + (tezCounters == null ? "null" : tezCounters.toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ")); }
@Override public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException, TezException { // Keep-alive information. The client should be informed and will have to take care of // re-submitting the work. // Some parts of fault tolerance go here. // This also provides completion information, and a possible notification when task actually // starts running (first heartbeat) if (LOG.isDebugEnabled()) { LOG.debug("Received heartbeat from container, request=" + request); } // Incoming events can be ignored until the point when shuffle needs to be handled, instead of // just scans. TezHeartbeatResponse response = new TezHeartbeatResponse(); response.setLastRequestId(request.getRequestId()); // Assuming TaskAttemptId and FragmentIdentifierString are the same. Verify this. TezTaskAttemptID taskAttemptId = request.getCurrentTaskAttemptID(); String taskAttemptIdString = taskAttemptId.toString(); updateHeartbeatInfo(taskAttemptIdString); List<TezEvent> tezEvents = null; PendingEventData pendingEventData = pendingEvents.remove(taskAttemptIdString); if (pendingEventData == null) { tezEvents = Collections.emptyList(); // If this heartbeat was not from a pending event and it's not in our list of registered // tasks, if (!registeredTasks.containsKey(taskAttemptIdString)) { LOG.info("Unexpected heartbeat from " + taskAttemptIdString); response.setShouldDie(); // Do any of the other fields need to be set? return response; } } else { tezEvents = pendingEventData.tezEvents; // Tasks removed from the pending list should then be added to the registered list. registeredTasks.put(taskAttemptIdString, pendingEventData.heartbeatInfo); } response.setLastRequestId(request.getRequestId()); // Irrelevant from eventIds. This can be tracked in the AM itself, instead of polluting the // task. // Also since we have all the MRInput events here - they'll all be sent in together. response.setNextFromEventId(0); // Irrelevant. See comment above. response.setNextPreRoutedEventId(0); // Irrelevant. See comment above. response.setEvents(tezEvents); List<TezEvent> inEvents = request.getEvents(); if (LOG.isDebugEnabled()) { LOG.debug( "Heartbeat from " + taskAttemptIdString + " events: " + (inEvents != null ? inEvents.size() : -1)); } for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) { EventType eventType = tezEvent.getEventType(); switch (eventType) { case TASK_ATTEMPT_COMPLETED_EVENT: LOG.debug("Task completed event for " + taskAttemptIdString); registeredTasks.remove(taskAttemptIdString); break; case TASK_ATTEMPT_FAILED_EVENT: LOG.debug("Task failed event for " + taskAttemptIdString); registeredTasks.remove(taskAttemptIdString); break; case TASK_STATUS_UPDATE_EVENT: // If we want to handle counters LOG.debug("Task update event for " + taskAttemptIdString); break; default: LOG.warn("Unhandled event type " + eventType); break; } } // Pass the request on to the responder try { if (responder != null) { responder.heartbeat(request); } } catch (Exception err) { LOG.error("Error during responder execution", err); } return response; }
@SuppressWarnings("unchecked") @Test public void testCredentialsTransfer() { WrappedContainerMultipleDAGs wc = new WrappedContainerMultipleDAGs(); TezDAGID dagID2 = TezDAGID.getInstance("800", 500, 2); TezDAGID dagID3 = TezDAGID.getInstance("800", 500, 3); TezVertexID vertexID2 = TezVertexID.getInstance(dagID2, 1); TezVertexID vertexID3 = TezVertexID.getInstance(dagID3, 1); TezTaskID taskID2 = TezTaskID.getInstance(vertexID2, 1); TezTaskID taskID3 = TezTaskID.getInstance(vertexID3, 1); TezTaskAttemptID attempt11 = TezTaskAttemptID.getInstance(wc.taskID, 200); TezTaskAttemptID attempt12 = TezTaskAttemptID.getInstance(wc.taskID, 300); TezTaskAttemptID attempt21 = TezTaskAttemptID.getInstance(taskID2, 200); TezTaskAttemptID attempt22 = TezTaskAttemptID.getInstance(taskID2, 300); TezTaskAttemptID attempt31 = TezTaskAttemptID.getInstance(taskID3, 200); TezTaskAttemptID attempt32 = TezTaskAttemptID.getInstance(taskID3, 300); Map<String, LocalResource> LRs = new HashMap<String, LocalResource>(); AMContainerTask fetchedTask = null; Token<TokenIdentifier> amGenToken = mock(Token.class); Token<TokenIdentifier> token1 = mock(Token.class); Token<TokenIdentifier> token3 = mock(Token.class); Credentials containerCredentials = new Credentials(); TokenCache.setSessionToken(amGenToken, containerCredentials); Text token1Name = new Text("tokenDag1"); Text token3Name = new Text("tokenDag3"); Credentials dag1Credentials = new Credentials(); dag1Credentials.addToken(new Text(token1Name), token1); Credentials dag3Credentials = new Credentials(); dag3Credentials.addToken(new Text(token3Name), token3); wc.launchContainer(new HashMap<String, LocalResource>(), containerCredentials); wc.containerLaunched(); wc.assignTaskAttempt(attempt11, LRs, dag1Credentials); fetchedTask = wc.pullTaskToRun(); assertTrue(fetchedTask.haveCredentialsChanged()); assertNotNull(fetchedTask.getCredentials()); assertNotNull(fetchedTask.getCredentials().getToken(token1Name)); wc.taskAttemptSucceeded(attempt11); wc.assignTaskAttempt(attempt12, LRs, dag1Credentials); fetchedTask = wc.pullTaskToRun(); assertFalse(fetchedTask.haveCredentialsChanged()); assertNull(fetchedTask.getCredentials()); wc.taskAttemptSucceeded(attempt12); // Move to running a second DAG, with no credentials. wc.setNewDAGID(dagID2); wc.assignTaskAttempt(attempt21, LRs, null); fetchedTask = wc.pullTaskToRun(); assertTrue(fetchedTask.haveCredentialsChanged()); assertNull(fetchedTask.getCredentials()); wc.taskAttemptSucceeded(attempt21); wc.assignTaskAttempt(attempt22, LRs, null); fetchedTask = wc.pullTaskToRun(); assertFalse(fetchedTask.haveCredentialsChanged()); assertNull(fetchedTask.getCredentials()); wc.taskAttemptSucceeded(attempt22); // Move to running a third DAG, with Credentials this time wc.setNewDAGID(dagID3); wc.assignTaskAttempt(attempt31, LRs, dag3Credentials); fetchedTask = wc.pullTaskToRun(); assertTrue(fetchedTask.haveCredentialsChanged()); assertNotNull(fetchedTask.getCredentials()); assertNotNull(fetchedTask.getCredentials().getToken(token3Name)); assertNull(fetchedTask.getCredentials().getToken(token1Name)); wc.taskAttemptSucceeded(attempt31); wc.assignTaskAttempt(attempt32, LRs, dag1Credentials); fetchedTask = wc.pullTaskToRun(); assertFalse(fetchedTask.haveCredentialsChanged()); assertNull(fetchedTask.getCredentials()); wc.taskAttemptSucceeded(attempt32); }
private TezTaskAttemptID getNewTaskAttemptID(TezTaskID taskId) { return TezTaskAttemptID.getInstance(taskId, taskAttemptCounter++); }