public void resetForNewExecution() { if (!(numSubtasksInFinalState == 0 || numSubtasksInFinalState == parallelism)) { throw new IllegalStateException("Cannot reset vertex that is not in final state"); } synchronized (stateMonitor) { // check and reset the sharing groups with scheduler hints if (slotSharingGroup != null) { slotSharingGroup.clearTaskAssignment(); } // reset vertices one by one. if one reset fails, the "vertices in final state" // fields will be consistent to handle triggered cancel calls for (int i = 0; i < parallelism; i++) { taskVertices[i].resetForNewExecution(); if (finishedSubtasks[i]) { finishedSubtasks[i] = false; numSubtasksInFinalState--; } } if (numSubtasksInFinalState != 0) { throw new RuntimeException("Bug: resetting the execution job vertex failed."); } // set up the input splits again try { if (this.inputSplits != null) { // lazy assignment @SuppressWarnings("unchecked") InputSplitSource<InputSplit> splitSource = (InputSplitSource<InputSplit>) jobVertex.getInputSplitSource(); this.splitAssigner = splitSource.getInputSplitAssigner(this.inputSplits); } } catch (Throwable t) { throw new RuntimeException( "Re-creating the input split assigner failed: " + t.getMessage(), t); } // Reset intermediate results for (IntermediateResult result : producedDataSets) { result.resetForNewExecution(); } } }
public ExecutionJobVertex( ExecutionGraph graph, JobVertex jobVertex, int defaultParallelism, Time timeout, long createTimestamp) throws JobException, IOException { if (graph == null || jobVertex == null) { throw new NullPointerException(); } this.graph = graph; this.jobVertex = jobVertex; int vertexParallelism = jobVertex.getParallelism(); int numTaskVertices = vertexParallelism > 0 ? vertexParallelism : defaultParallelism; this.parallelism = numTaskVertices; int maxP = jobVertex.getMaxParallelism(); Preconditions.checkArgument( maxP >= parallelism, "The maximum parallelism (" + maxP + ") must be greater or equal than the parallelism (" + parallelism + ")."); this.maxParallelism = maxP; this.serializedTaskInformation = new SerializedValue<>( new TaskInformation( jobVertex.getID(), jobVertex.getName(), parallelism, maxParallelism, jobVertex.getInvokableClassName(), jobVertex.getConfiguration())); this.taskVertices = new ExecutionVertex[numTaskVertices]; this.inputs = new ArrayList<IntermediateResult>(jobVertex.getInputs().size()); // take the sharing group this.slotSharingGroup = jobVertex.getSlotSharingGroup(); this.coLocationGroup = jobVertex.getCoLocationGroup(); // setup the coLocation group if (coLocationGroup != null && slotSharingGroup == null) { throw new JobException("Vertex uses a co-location constraint without using slot sharing"); } // create the intermediate results this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()]; for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) { final IntermediateDataSet result = jobVertex.getProducedDataSets().get(i); this.producedDataSets[i] = new IntermediateResult(result.getId(), this, numTaskVertices, result.getResultType()); } Configuration jobConfiguration = graph.getJobConfiguration(); int maxPriorAttemptsHistoryLength = jobConfiguration != null ? jobConfiguration.getInteger(JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE) : JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue(); // create all task vertices for (int i = 0; i < numTaskVertices; i++) { ExecutionVertex vertex = new ExecutionVertex( this, i, this.producedDataSets, timeout, createTimestamp, maxPriorAttemptsHistoryLength); this.taskVertices[i] = vertex; } // sanity check for the double referencing between intermediate result partitions and execution // vertices for (IntermediateResult ir : this.producedDataSets) { if (ir.getNumberOfAssignedPartitions() != parallelism) { throw new RuntimeException( "The intermediate result's partitions were not correctly assigned."); } } // set up the input splits, if the vertex has any try { @SuppressWarnings("unchecked") InputSplitSource<InputSplit> splitSource = (InputSplitSource<InputSplit>) jobVertex.getInputSplitSource(); if (splitSource != null) { Thread currentThread = Thread.currentThread(); ClassLoader oldContextClassLoader = currentThread.getContextClassLoader(); currentThread.setContextClassLoader(graph.getUserClassLoader()); try { inputSplits = splitSource.createInputSplits(numTaskVertices); if (inputSplits != null) { splitAssigner = splitSource.getInputSplitAssigner(inputSplits); } } finally { currentThread.setContextClassLoader(oldContextClassLoader); } } else { inputSplits = null; } } catch (Throwable t) { throw new JobException("Creating the input splits caused an error: " + t.getMessage(), t); } finishedSubtasks = new boolean[parallelism]; }