private JobGraph createTestJobGraph( String jobName, int senderParallelism, int receiverParallelism) { // The sender and receiver invokable logic ensure that each subtask gets the expected data final JobVertex sender = new JobVertex("Sender"); sender.setInvokableClass(RoundRobinSubtaskIndexSender.class); sender .getConfiguration() .setInteger(RoundRobinSubtaskIndexSender.CONFIG_KEY, receiverParallelism); sender.setParallelism(senderParallelism); final JobVertex receiver = new JobVertex("Receiver"); receiver.setInvokableClass(SubtaskIndexReceiver.class); receiver.getConfiguration().setInteger(SubtaskIndexReceiver.CONFIG_KEY, senderParallelism); receiver.setParallelism(receiverParallelism); receiver.connectNewDataSetAsInput( sender, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); final JobGraph jobGraph = new JobGraph(jobName, sender, receiver); // We need to allow queued scheduling, because there are not enough slots available // to run all tasks at once. We queue tasks and then let them finish/consume the blocking // result one after the other. jobGraph.setAllowQueuedScheduling(true); return jobGraph; }
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]; }
public static JobGraph getJobGraph(String[] args) throws Exception { int parallelism = 2; String pageWithRankInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank"; String adjacencyListInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + // "test-inputs/danglingpagerank/adjacencylists"; String outputPath = OperatingSystem.isWindows() ? "file:/c:/tmp/flink/iterations" : "file:///tmp/flink/iterations"; // String confPath = PlayConstants.PLAY_DIR + "local-conf"; int minorConsumer = 25; int matchMemory = 50; int coGroupSortMemory = 50; int numIterations = 25; long numVertices = 5; long numDanglingVertices = 1; String failingWorkers = "1"; int failingIteration = 2; double messageLoss = 0.75; if (args.length >= 15) { parallelism = Integer.parseInt(args[0]); pageWithRankInputPath = args[1]; adjacencyListInputPath = args[2]; outputPath = args[3]; // confPath = args[4]; minorConsumer = Integer.parseInt(args[5]); matchMemory = Integer.parseInt(args[6]); coGroupSortMemory = Integer.parseInt(args[7]); numIterations = Integer.parseInt(args[8]); numVertices = Long.parseLong(args[9]); numDanglingVertices = Long.parseLong(args[10]); failingWorkers = args[11]; failingIteration = Integer.parseInt(args[12]); messageLoss = Double.parseDouble(args[13]); } int totalMemoryConsumption = 3 * minorConsumer + matchMemory + coGroupSortMemory; JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank"); // --------------- the inputs --------------------- // page rank input InputFormatVertex pageWithRankInput = JobGraphUtils.createInput( new ImprovedDanglingPageRankInputFormat(), pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism); TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration()); pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); pageWithRankInputConfig.setOutputComparator(fieldZeroComparator, 0); pageWithRankInputConfig.setOutputSerializer(recSerializer); pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices)); // edges as adjacency list InputFormatVertex adjacencyListInput = JobGraphUtils.createInput( new ImprovedAdjacencyListInputFormat(), adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism); TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration()); adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); adjacencyListInputConfig.setOutputSerializer(recSerializer); adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0); // --------------- the head --------------------- JobVertex head = JobGraphUtils.createTask( IterationHeadPactTask.class, "IterationHead", jobGraph, parallelism); TaskConfig headConfig = new TaskConfig(head.getConfiguration()); headConfig.setIterationId(ITERATION_ID); // initial input / partial solution headConfig.addInputToGroup(0); headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0); headConfig.setInputSerializer(recSerializer, 0); headConfig.setInputComparator(fieldZeroComparator, 0); headConfig.setInputLocalStrategy(0, LocalStrategy.SORT); headConfig.setRelativeMemoryInput(0, (double) minorConsumer / totalMemoryConsumption); headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT); headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD); // back channel / iterations headConfig.setRelativeBackChannelMemory((double) minorConsumer / totalMemoryConsumption); // output into iteration headConfig.setOutputSerializer(recSerializer); headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD); headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD); // final output TaskConfig headFinalOutConfig = new TaskConfig(new Configuration()); headFinalOutConfig.setOutputSerializer(recSerializer); headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD); headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig); // the sync headConfig.setIterationHeadIndexOfSyncOutput(3); headConfig.setNumberOfIterations(numIterations); // the driver headConfig.setDriver(CollectorMapDriver.class); headConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP); headConfig.setStubWrapper(new UserCodeClassWrapper<CompensatingMap>(CompensatingMap.class)); headConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices)); headConfig.setStubParameter("compensation.failingWorker", failingWorkers); headConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration)); headConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss)); headConfig.addIterationAggregator( CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator()); // --------------- the join --------------------- JobVertex intermediate = JobGraphUtils.createTask( IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, parallelism); TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration()); intermediateConfig.setIterationId(ITERATION_ID); // intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class); intermediateConfig.setDriver(BuildSecondCachedMatchDriver.class); intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND); intermediateConfig.setRelativeMemoryDriver((double) matchMemory / totalMemoryConsumption); intermediateConfig.addInputToGroup(0); intermediateConfig.addInputToGroup(1); intermediateConfig.setInputSerializer(recSerializer, 0); intermediateConfig.setInputSerializer(recSerializer, 1); intermediateConfig.setDriverComparator(fieldZeroComparator, 0); intermediateConfig.setDriverComparator(fieldZeroComparator, 1); intermediateConfig.setDriverPairComparator(pairComparatorFactory); intermediateConfig.setOutputSerializer(recSerializer); intermediateConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH); intermediateConfig.setOutputComparator(fieldZeroComparator, 0); intermediateConfig.setStubWrapper( new UserCodeClassWrapper<CompensatableDotProductMatch>(CompensatableDotProductMatch.class)); intermediateConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices)); intermediateConfig.setStubParameter("compensation.failingWorker", failingWorkers); intermediateConfig.setStubParameter( "compensation.failingIteration", String.valueOf(failingIteration)); intermediateConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss)); // ---------------- the tail (co group) -------------------- JobVertex tail = JobGraphUtils.createTask( IterationTailPactTask.class, "IterationTail", jobGraph, parallelism); TaskConfig tailConfig = new TaskConfig(tail.getConfiguration()); tailConfig.setIterationId(ITERATION_ID); tailConfig.setIsWorksetUpdate(); // TODO we need to combine! // inputs and driver tailConfig.setDriver(CoGroupDriver.class); tailConfig.setDriverStrategy(DriverStrategy.CO_GROUP); tailConfig.addInputToGroup(0); tailConfig.addInputToGroup(1); tailConfig.setInputSerializer(recSerializer, 0); tailConfig.setInputSerializer(recSerializer, 1); tailConfig.setDriverComparator(fieldZeroComparator, 0); tailConfig.setDriverComparator(fieldZeroComparator, 1); tailConfig.setDriverPairComparator(pairComparatorFactory); tailConfig.setInputAsynchronouslyMaterialized(0, true); tailConfig.setRelativeInputMaterializationMemory( 0, (double) minorConsumer / totalMemoryConsumption); tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT); tailConfig.setInputComparator(fieldZeroComparator, 1); tailConfig.setRelativeMemoryInput(1, (double) coGroupSortMemory / totalMemoryConsumption); tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT); tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD); // output tailConfig.setOutputSerializer(recSerializer); // the stub tailConfig.setStubWrapper( new UserCodeClassWrapper<CompensatableDotProductCoGroup>( CompensatableDotProductCoGroup.class)); tailConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices)); tailConfig.setStubParameter( "pageRank.numDanglingVertices", String.valueOf(numDanglingVertices)); tailConfig.setStubParameter("compensation.failingWorker", failingWorkers); tailConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration)); tailConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss)); // --------------- the output --------------------- OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism); TaskConfig outputConfig = new TaskConfig(output.getConfiguration()); outputConfig.addInputToGroup(0); outputConfig.setInputSerializer(recSerializer, 0); outputConfig.setStubWrapper( new UserCodeClassWrapper<PageWithRankOutFormat>(PageWithRankOutFormat.class)); outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath); // --------------- the auxiliaries --------------------- JobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism); TaskConfig syncConfig = new TaskConfig(sync.getConfiguration()); syncConfig.setNumberOfIterations(numIterations); syncConfig.addIterationAggregator( CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator()); syncConfig.setConvergenceCriterion( CompensatableDotProductCoGroup.AGGREGATOR_NAME, new DiffL1NormConvergenceCriterion()); syncConfig.setIterationId(ITERATION_ID); // --------------- the wiring --------------------- JobGraphUtils.connect(pageWithRankInput, head, DistributionPattern.ALL_TO_ALL); JobGraphUtils.connect(head, intermediate, DistributionPattern.POINTWISE); intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); JobGraphUtils.connect(adjacencyListInput, intermediate, DistributionPattern.ALL_TO_ALL); JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism); JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE); SlotSharingGroup sharingGroup = new SlotSharingGroup(); pageWithRankInput.setSlotSharingGroup(sharingGroup); adjacencyListInput.setSlotSharingGroup(sharingGroup); head.setSlotSharingGroup(sharingGroup); intermediate.setSlotSharingGroup(sharingGroup); tail.setSlotSharingGroup(sharingGroup); output.setSlotSharingGroup(sharingGroup); sync.setSlotSharingGroup(sharingGroup); tail.setStrictlyCoLocatedWith(head); intermediate.setStrictlyCoLocatedWith(head); return jobGraph; }