adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
adjacencyListInputConfig.setOutputSerializer(recSerializer);
adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0);
// --------------- the head ---------------------
JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
degreeOfParallelism, numSubTasksPerInstance);
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.setMemoryInput(0, minorConsumer * JobGraphUtils.MEGABYTE);
headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
// back channel / iterations
headConfig.setBackChannelMemory(minorConsumer * JobGraphUtils.MEGABYTE);
// 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, PageRankStatsAggregator.class);
// --------------- the join ---------------------
JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
"IterationIntermediate", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
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.setMemoryDriver(matchMemory * JobGraphUtils.MEGABYTE);
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) --------------------
JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
degreeOfParallelism, numSubTasksPerInstance);
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.setInputMaterializationMemory(0, minorConsumer * JobGraphUtils.MEGABYTE);
tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
tailConfig.setInputComparator(fieldZeroComparator, 1);
tailConfig.setMemoryInput(1, coGroupSortMemory * JobGraphUtils.MEGABYTE);
tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
// output
tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
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 ---------------------
JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism,
numSubTasksPerInstance);
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 ---------------------
JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
degreeOfParallelism, numSubTasksPerInstance);
JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
syncConfig.setNumberOfIterations(numIterations);
syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, PageRankStatsAggregator.class);
syncConfig.setConvergenceCriterion(CompensatableDotProductCoGroup.AGGREGATOR_NAME, DiffL1NormConvergenceCriterion.class);
syncConfig.setIterationId(ITERATION_ID);
// --------------- the wiring ---------------------
JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
JobGraphUtils.connect(head, tail, ChannelType.NETWORK, DistributionPattern.POINTWISE);
JobGraphUtils.connect(intermediate, tail, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
fakeTailOutput.setVertexToShareInstancesWith(tail);
tail.setVertexToShareInstancesWith(head);
pageWithRankInput.setVertexToShareInstancesWith(head);
adjacencyListInput.setVertexToShareInstancesWith(head);
intermediate.setVertexToShareInstancesWith(head);
output.setVertexToShareInstancesWith(head);
sync.setVertexToShareInstancesWith(head);