@Override public ExecutionState getAggregateState() { int[] num = new int[ExecutionState.values().length]; for (ExecutionVertex vertex : this.taskVertices) { num[vertex.getExecutionState().ordinal()]++; } return getAggregateJobVertexState(num, parallelism); }
public void scheduleAll(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException { ExecutionVertex[] vertices = this.taskVertices; // kick off the tasks for (ExecutionVertex ev : vertices) { ev.scheduleForExecution(slotProvider, queued); } }
public void connectToPredecessors( Map<IntermediateDataSetID, IntermediateResult> intermediateDataSets) throws JobException { List<JobEdge> inputs = jobVertex.getInputs(); if (LOG.isDebugEnabled()) { LOG.debug( String.format( "Connecting ExecutionJobVertex %s (%s) to %d predecessors.", jobVertex.getID(), jobVertex.getName(), inputs.size())); } for (int num = 0; num < inputs.size(); num++) { JobEdge edge = inputs.get(num); if (LOG.isDebugEnabled()) { if (edge.getSource() == null) { LOG.debug( String.format( "Connecting input %d of vertex %s (%s) to intermediate result referenced via ID %s.", num, jobVertex.getID(), jobVertex.getName(), edge.getSourceId())); } else { LOG.debug( String.format( "Connecting input %d of vertex %s (%s) to intermediate result referenced via predecessor %s (%s).", num, jobVertex.getID(), jobVertex.getName(), edge.getSource().getProducer().getID(), edge.getSource().getProducer().getName())); } } // fetch the intermediate result via ID. if it does not exist, then it either has not been // created, or the order // in which this method is called for the job vertices is not a topological order IntermediateResult ires = intermediateDataSets.get(edge.getSourceId()); if (ires == null) { throw new JobException( "Cannot connect this job graph to the previous graph. No previous intermediate result found for ID " + edge.getSourceId()); } this.inputs.add(ires); int consumerIndex = ires.registerConsumer(); for (int i = 0; i < parallelism; i++) { ExecutionVertex ev = taskVertices[i]; ev.connectSource(num, ires, edge, consumerIndex); } } }
void markFinished( Map<AccumulatorRegistry.Metric, Accumulator<?, ?>> flinkAccumulators, Map<String, Accumulator<?, ?>> userAccumulators) { // this call usually comes during RUNNING, but may also come while still in deploying (very fast // tasks!) while (true) { ExecutionState current = this.state; if (current == RUNNING || current == DEPLOYING) { if (transitionState(current, FINISHED)) { try { for (IntermediateResultPartition finishedPartition : getVertex().finishAllBlockingPartitions()) { IntermediateResultPartition[] allPartitions = finishedPartition.getIntermediateResult().getPartitions(); for (IntermediateResultPartition partition : allPartitions) { scheduleOrUpdateConsumers(partition.getConsumers()); } } synchronized (accumulatorLock) { this.flinkAccumulators = flinkAccumulators; this.userAccumulators = userAccumulators; } assignedResource.releaseSlot(); vertex.getExecutionGraph().deregisterExecution(this); } finally { vertex.executionFinished(); } return; } } else if (current == CANCELING) { // we sent a cancel call, and the task manager finished before it arrived. We // will never get a CANCELED call back from the job manager cancelingComplete(); return; } else if (current == CANCELED || current == FAILED) { if (LOG.isDebugEnabled()) { LOG.debug("Task FINISHED, but concurrently went to state " + state); } return; } else { // this should not happen, we need to fail this markFailed(new Exception("Vertex received FINISHED message while being in state " + state)); return; } } }
public void cancel() { // depending on the previous state, we go directly to cancelled (no cancel call necessary) // -- or to canceling (cancel call needs to be sent to the task manager) // because of several possibly previous states, we need to again loop until we make a // successful atomic state transition while (true) { ExecutionState current = this.state; if (current == CANCELING || current == CANCELED) { // already taken care of, no need to cancel again return; } // these two are the common cases where we need to send a cancel call else if (current == RUNNING || current == DEPLOYING) { // try to transition to canceling, if successful, send the cancel call if (transitionState(current, CANCELING)) { sendCancelRpcCall(); return; } // else: fall through the loop } else if (current == FINISHED || current == FAILED) { // nothing to do any more. finished failed before it could be cancelled. // in any case, the task is removed from the TaskManager already sendFailIntermediateResultPartitionsRpcCall(); return; } else if (current == CREATED || current == SCHEDULED) { // from here, we can directly switch to cancelled, because the no task has been deployed if (transitionState(current, CANCELED)) { // we skip the canceling state. set the timestamp, for a consistent appearance markTimestamp(CANCELING, getStateTimestamp(CANCELED)); try { vertex.getExecutionGraph().deregisterExecution(this); if (assignedResource != null) { assignedResource.releaseSlot(); } } finally { vertex.executionCanceled(); } return; } // else: fall through the loop } else { throw new IllegalStateException(current.name()); } } }
public StringifiedAccumulatorResult[] getAggregatedUserAccumulatorsStringified() { Map<String, Accumulator<?, ?>> userAccumulators = new HashMap<String, Accumulator<?, ?>>(); for (ExecutionVertex vertex : taskVertices) { Map<String, Accumulator<?, ?>> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); if (next != null) { AccumulatorHelper.mergeInto(userAccumulators, next); } } return StringifiedAccumulatorResult.stringifyAccumulatorResults(userAccumulators); }
private boolean transitionState( ExecutionState currentState, ExecutionState targetState, Throwable error) { if (STATE_UPDATER.compareAndSet(this, currentState, targetState)) { markTimestamp(targetState); LOG.info( getVertex().getTaskNameWithSubtaskIndex() + " (" + getAttemptId() + ") switched from " + currentState + " to " + targetState); // make sure that the state transition completes normally. // potential errors (in listeners may not affect the main logic) try { vertex.notifyStateTransition(attemptId, targetState, error); } catch (Throwable t) { LOG.error("Error while notifying execution graph of execution state transition.", t); } return true; } else { return false; } }
@Override public String toString() { return String.format( "Attempt #%d (%s) @ %s - [%s]", attemptNumber, vertex.getSimpleName(), (assignedResource == null ? "(unassigned)" : assignedResource.toString()), state); }
void cancelingComplete() { // the taskmanagers can themselves cancel tasks without an external trigger, if they find that // the // network stack is canceled (for example by a failing / canceling receiver or sender // this is an artifact of the old network runtime, but for now we need to support task // transitions // from running directly to canceled while (true) { ExecutionState current = this.state; if (current == CANCELED) { return; } else if (current == CANCELING || current == RUNNING || current == DEPLOYING) { if (transitionState(current, CANCELED)) { try { assignedResource.releaseSlot(); vertex.getExecutionGraph().deregisterExecution(this); } finally { vertex.executionCanceled(); } return; } // else fall through the loop } else { // failing in the meantime may happen and is no problem. // anything else is a serious problem !!! if (current != FAILED) { String message = String.format( "Asynchronous race: Found state %s after successful cancel call.", state); LOG.error(message); vertex.getExecutionGraph().fail(new Exception(message)); } return; } } }
public void fail(Throwable t) { for (ExecutionVertex ev : getTaskVertices()) { ev.fail(t); } }
public void cancel() { for (ExecutionVertex ev : getTaskVertices()) { ev.cancel(); } }
private boolean processFail(Throwable t, boolean isCallback) { // damn, we failed. This means only that we keep our books and notify our parent // JobExecutionVertex // the actual computation on the task manager is cleaned up by the TaskManager that noticed the // failure // we may need to loop multiple times (in the presence of concurrent calls) in order to // atomically switch to failed while (true) { ExecutionState current = this.state; if (current == FAILED) { // already failed. It is enough to remember once that we failed (its sad enough) return false; } if (current == CANCELED) { // we are already aborting or are already aborted if (LOG.isDebugEnabled()) { LOG.debug( String.format( "Ignoring transition of vertex %s to %s while being %s", getVertexWithAttempt(), FAILED, CANCELED)); } return false; } if (transitionState(current, FAILED, t)) { // success (in a manner of speaking) this.failureCause = t; try { if (assignedResource != null) { assignedResource.releaseSlot(); } vertex.getExecutionGraph().deregisterExecution(this); } finally { vertex.executionFailed(t); } if (!isCallback && (current == RUNNING || current == DEPLOYING)) { if (LOG.isDebugEnabled()) { LOG.debug("Sending out cancel request, to remove task execution from TaskManager."); } try { if (assignedResource != null) { sendCancelRpcCall(); } } catch (Throwable tt) { // no reason this should ever happen, but log it to be safe LOG.error("Error triggering cancel call while marking task as failed.", tt); } } // leave the loop return true; } } }
void scheduleOrUpdateConsumers(List<List<ExecutionEdge>> allConsumers) { final int numConsumers = allConsumers.size(); if (numConsumers > 1) { fail( new IllegalStateException( "Currently, only a single consumer group per partition is supported.")); } else if (numConsumers == 0) { return; } for (ExecutionEdge edge : allConsumers.get(0)) { final ExecutionVertex consumerVertex = edge.getTarget(); final Execution consumer = consumerVertex.getCurrentExecutionAttempt(); final ExecutionState consumerState = consumer.getState(); final IntermediateResultPartition partition = edge.getSource(); // ---------------------------------------------------------------- // Consumer is created => try to deploy and cache input channel // descriptors if there is a deployment race // ---------------------------------------------------------------- if (consumerState == CREATED) { final Execution partitionExecution = partition.getProducer().getCurrentExecutionAttempt(); consumerVertex.cachePartitionInfo( PartialInputChannelDeploymentDescriptor.fromEdge(partition, partitionExecution)); // When deploying a consuming task, its task deployment descriptor will contain all // deployment information available at the respective time. It is possible that some // of the partitions to be consumed have not been created yet. These are updated // runtime via the update messages. // // TODO The current approach may send many update messages even though the consuming // task has already been deployed with all necessary information. We have to check // whether this is a problem and fix it, if it is. future( new Callable<Boolean>() { @Override public Boolean call() throws Exception { try { consumerVertex.scheduleForExecution( consumerVertex.getExecutionGraph().getScheduler(), consumerVertex.getExecutionGraph().isQueuedSchedulingAllowed()); } catch (Throwable t) { fail( new IllegalStateException( "Could not schedule consumer " + "vertex " + consumerVertex, t)); } return true; } }, executionContext); // double check to resolve race conditions if (consumerVertex.getExecutionState() == RUNNING) { consumerVertex.sendPartitionInfos(); } } // ---------------------------------------------------------------- // Consumer is running => send update message now // ---------------------------------------------------------------- else { if (consumerState == RUNNING) { final SimpleSlot consumerSlot = consumer.getAssignedResource(); if (consumerSlot == null) { // The consumer has been reset concurrently continue; } final Instance consumerInstance = consumerSlot.getInstance(); final ResultPartitionID partitionId = new ResultPartitionID(partition.getPartitionId(), attemptId); final Instance partitionInstance = partition.getProducer().getCurrentAssignedResource().getInstance(); final ResultPartitionLocation partitionLocation; if (consumerInstance.equals(partitionInstance)) { // Consuming task is deployed to the same instance as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } else { // Different instances => remote final ConnectionID connectionId = new ConnectionID( partitionInstance.getInstanceConnectionInfo(), partition.getIntermediateResult().getConnectionIndex()); partitionLocation = ResultPartitionLocation.createRemote(connectionId); } final InputChannelDeploymentDescriptor descriptor = new InputChannelDeploymentDescriptor(partitionId, partitionLocation); final UpdatePartitionInfo updateTaskMessage = new UpdateTaskSinglePartitionInfo( consumer.getAttemptId(), partition.getIntermediateResult().getId(), descriptor); sendUpdatePartitionInfoRpcCall(consumerSlot, updateTaskMessage); } // ---------------------------------------------------------------- // Consumer is scheduled or deploying => cache input channel // deployment descriptors and send update message later // ---------------------------------------------------------------- else if (consumerState == SCHEDULED || consumerState == DEPLOYING) { final Execution partitionExecution = partition.getProducer().getCurrentExecutionAttempt(); consumerVertex.cachePartitionInfo( PartialInputChannelDeploymentDescriptor.fromEdge(partition, partitionExecution)); // double check to resolve race conditions if (consumerVertex.getExecutionState() == RUNNING) { consumerVertex.sendPartitionInfos(); } } } } }
public void deployToSlot(final SimpleSlot slot) throws JobException { // sanity checks if (slot == null) { throw new NullPointerException(); } if (!slot.isAlive()) { throw new JobException("Target slot for deployment is not alive."); } // make sure exactly one deployment call happens from the correct state // note: the transition from CREATED to DEPLOYING is for testing purposes only ExecutionState previous = this.state; if (previous == SCHEDULED || previous == CREATED) { if (!transitionState(previous, DEPLOYING)) { // race condition, someone else beat us to the deploying call. // this should actually not happen and indicates a race somewhere else throw new IllegalStateException("Cannot deploy task: Concurrent deployment call race."); } } else { // vertex may have been cancelled, or it was already scheduled throw new IllegalStateException( "The vertex must be in CREATED or SCHEDULED state to be deployed. Found state " + previous); } try { // good, we are allowed to deploy if (!slot.setExecutedVertex(this)) { throw new JobException("Could not assign the ExecutionVertex to the slot " + slot); } this.assignedResource = slot; this.assignedResourceLocation = slot.getInstance().getInstanceConnectionInfo(); // race double check, did we fail/cancel and do we need to release the slot? if (this.state != DEPLOYING) { slot.releaseSlot(); return; } if (LOG.isInfoEnabled()) { LOG.info( String.format( "Deploying %s (attempt #%d) to %s", vertex.getSimpleName(), attemptNumber, slot.getInstance().getInstanceConnectionInfo().getHostname())); } final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor( attemptId, slot, operatorState, recoveryTimestamp, attemptNumber); // register this execution at the execution graph, to receive call backs vertex.getExecutionGraph().registerExecution(this); final Instance instance = slot.getInstance(); final ActorGateway gateway = instance.getActorGateway(); final Future<Object> deployAction = gateway.ask(new SubmitTask(deployment), timeout); deployAction.onComplete( new OnComplete<Object>() { @Override public void onComplete(Throwable failure, Object success) throws Throwable { if (failure != null) { if (failure instanceof TimeoutException) { String taskname = deployment.getTaskInfo().getTaskNameWithSubtasks() + " (" + attemptId + ')'; markFailed( new Exception( "Cannot deploy task " + taskname + " - TaskManager (" + instance + ") not responding after a timeout of " + timeout, failure)); } else { markFailed(failure); } } else { if (!(success.equals(Messages.getAcknowledge()))) { markFailed( new Exception( "Failed to deploy the task to slot " + slot + ": Response was not of type Acknowledge")); } } } }, executionContext); } catch (Throwable t) { markFailed(t); ExceptionUtils.rethrow(t); } }
/** * NOTE: This method only throws exceptions if it is in an illegal state to be scheduled, or if * the tasks needs to be scheduled immediately and no resource is available. If the task is * accepted by the schedule, any error sets the vertex state to failed and triggers the recovery * logic. * * @param scheduler The scheduler to use to schedule this execution attempt. * @param queued Flag to indicate whether the scheduler may queue this task if it cannot * immediately deploy it. * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only * state that permits scheduling. * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources * are currently available. */ public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException { if (scheduler == null) { throw new IllegalArgumentException("Cannot send null Scheduler when scheduling execution."); } final SlotSharingGroup sharingGroup = vertex.getJobVertex().getSlotSharingGroup(); final CoLocationConstraint locationConstraint = vertex.getLocationConstraint(); // sanity check if (locationConstraint != null && sharingGroup == null) { throw new RuntimeException( "Trying to schedule with co-location constraint but without slot sharing allowed."); } if (transitionState(CREATED, SCHEDULED)) { ScheduledUnit toSchedule = locationConstraint == null ? new ScheduledUnit(this, sharingGroup) : new ScheduledUnit(this, sharingGroup, locationConstraint); // IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are // returned // in all cases where the deployment failed. we use many try {} finally {} clauses to // assure that if (queued) { SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule); future.setFutureAction( new SlotAllocationFutureAction() { @Override public void slotAllocated(SimpleSlot slot) { try { deployToSlot(slot); } catch (Throwable t) { try { slot.releaseSlot(); } finally { markFailed(t); } } } }); } else { SimpleSlot slot = scheduler.scheduleImmediately(toSchedule); try { deployToSlot(slot); } catch (Throwable t) { try { slot.releaseSlot(); } finally { markFailed(t); } } } return true; } else { // call race, already deployed, or already done return false; } }
@Test public void testBuildDeploymentDescriptor() { try { final JobID jobId = new JobID(); final JobVertexID jid1 = new JobVertexID(); final JobVertexID jid2 = new JobVertexID(); final JobVertexID jid3 = new JobVertexID(); final JobVertexID jid4 = new JobVertexID(); JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); JobVertex v3 = new JobVertex("v3", jid3); JobVertex v4 = new JobVertex("v4", jid4); v1.setParallelism(10); v2.setParallelism(10); v3.setParallelism(10); v4.setParallelism(10); v1.setInvokableClass(BatchTask.class); v2.setInvokableClass(BatchTask.class); v3.setInvokableClass(BatchTask.class); v4.setInvokableClass(BatchTask.class); v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL); v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL); v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobId, "some job", new Configuration(), new SerializedValue<>(new ExecutionConfig()), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); List<JobVertex> ordered = Arrays.asList(v1, v2, v3, v4); eg.attachJobGraph(ordered); ExecutionJobVertex ejv = eg.getAllVertices().get(jid2); ExecutionVertex vertex = ejv.getTaskVertices()[3]; ExecutionGraphTestUtils.SimpleActorGateway instanceGateway = new ExecutionGraphTestUtils.SimpleActorGateway(TestingUtils.directExecutionContext()); final Instance instance = getInstance(instanceGateway); final SimpleSlot slot = instance.allocateSimpleSlot(jobId); assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); vertex.deployToSlot(slot); assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState()); TaskDeploymentDescriptor descr = instanceGateway.lastTDD; assertNotNull(descr); assertEquals(jobId, descr.getJobID()); assertEquals(jid2, descr.getVertexID()); assertEquals(3, descr.getIndexInSubtaskGroup()); assertEquals(10, descr.getNumberOfSubtasks()); assertEquals(BatchTask.class.getName(), descr.getInvokableClassName()); assertEquals("v2", descr.getTaskName()); List<ResultPartitionDeploymentDescriptor> producedPartitions = descr.getProducedPartitions(); List<InputGateDeploymentDescriptor> consumedPartitions = descr.getInputGates(); assertEquals(2, producedPartitions.size()); assertEquals(1, consumedPartitions.size()); assertEquals(10, producedPartitions.get(0).getNumberOfSubpartitions()); assertEquals(10, producedPartitions.get(1).getNumberOfSubpartitions()); assertEquals(10, consumedPartitions.get(0).getInputChannelDeploymentDescriptors().length); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
private static void addExecutionVertices( Map<ExecutionGroupVertex, ManagementGroupVertex> groupMap, ExecutionGraph executionGraph) { ExecutionGraphIterator iterator = new ExecutionGraphIterator(executionGraph, true); final Map<ExecutionVertex, ManagementVertex> vertexMap = new HashMap<ExecutionVertex, ManagementVertex>(); final Map<ExecutionGate, ManagementGate> gateMap = new HashMap<ExecutionGate, ManagementGate>(); while (iterator.hasNext()) { final ExecutionVertex ev = iterator.next(); final ManagementGroupVertex parent = groupMap.get(ev.getGroupVertex()); final AbstractInstance instance = ev.getAllocatedResource().getInstance(); final ManagementVertex managementVertex = new ManagementVertex( parent, ev.getID().toManagementVertexID(), (instance.getInstanceConnectionInfo() != null) ? instance.getInstanceConnectionInfo().toString() : instance.toString(), instance.getType().toString(), ev.getIndexInVertexGroup()); managementVertex.setExecutionState(ev.getExecutionState()); vertexMap.put(ev, managementVertex); for (int i = 0; i < ev.getNumberOfOutputGates(); i++) { final ExecutionGate outputGate = ev.getOutputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, new ManagementGateID(), i, false); gateMap.put(outputGate, managementGate); } for (int i = 0; i < ev.getNumberOfInputGates(); i++) { final ExecutionGate inputGate = ev.getInputGate(i); final ManagementGate managementGate = new ManagementGate(managementVertex, new ManagementGateID(), i, true); gateMap.put(inputGate, managementGate); } } iterator = new ExecutionGraphIterator(executionGraph, true); // Setup connections while (iterator.hasNext()) { final ExecutionVertex source = iterator.next(); for (int i = 0; i < source.getNumberOfOutputGates(); i++) { final ExecutionGate outputGate = source.getOutputGate(i); final ManagementGate manangementOutputGate = gateMap.get(outputGate); final ChannelType channelType = outputGate.getChannelType(); for (int j = 0; j < outputGate.getNumberOfEdges(); j++) { final ExecutionEdge outputChannel = outputGate.getEdge(j); final ManagementGate managementInputGate = gateMap.get(outputChannel.getInputGate()); final ManagementEdgeID sourceEdgeID = new ManagementEdgeID(outputChannel.getOutputChannelID()); final ManagementEdgeID targetEdgeID = new ManagementEdgeID(outputChannel.getInputChannelID()); new ManagementEdge( sourceEdgeID, targetEdgeID, manangementOutputGate, j, managementInputGate, outputChannel.getInputGateIndex(), channelType); } } } }
public String getVertexWithAttempt() { return vertex.getSimpleName() + " - execution #" + attemptNumber; }