JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
// --------------- the inputs ---------------------
// page rank input
InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
pageWithRankInputConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
// edges as adjacency list
InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
adjacencyListInputConfig.setOutputComparator(vertexWithAdjacencyListComparator, 0);
// --------------- the head ---------------------
AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
degreeOfParallelism);
TaskConfig headConfig = new TaskConfig(head.getConfiguration());
headConfig.setIterationId(ITERATION_ID);
// initial input / partial solution
headConfig.addInputToGroup(0);
headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
headConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
headConfig.setInputComparator(vertexWithRankAndDanglingComparator, 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(vertexWithRankAndDanglingSerializer);
headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
// final output
TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
headFinalOutConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
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<CustomCompensatingMap>(CustomCompensatingMap.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(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
// --------------- the join ---------------------
AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
"IterationIntermediate", jobGraph, degreeOfParallelism);
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(vertexWithRankAndDanglingSerializer, 0);
intermediateConfig.setInputSerializer(vertexWithAdjacencyListSerializer, 1);
intermediateConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
intermediateConfig.setDriverComparator(vertexWithAdjacencyListComparator, 1);
intermediateConfig.setDriverPairComparator(matchComparator);
intermediateConfig.setOutputSerializer(vertexWithRankSerializer);
intermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
intermediateConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductMatch>(CustomCompensatableDotProductMatch.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 combiner and the output
TaskConfig combinerConfig = new TaskConfig(new Configuration());
combinerConfig.addInputToGroup(0);
combinerConfig.setInputSerializer(vertexWithRankSerializer, 0);
combinerConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
combinerConfig.setDriverComparator(vertexWithRankComparator, 0);
combinerConfig.setDriverComparator(vertexWithRankComparator, 1);
combinerConfig.setRelativeMemoryDriver((double)coGroupSortMemory/totalMemoryConsumption);
combinerConfig.setOutputSerializer(vertexWithRankSerializer);
combinerConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
combinerConfig.setOutputComparator(vertexWithRankComparator, 0);
combinerConfig.setStubWrapper(new UserCodeClassWrapper<CustomRankCombiner>(CustomRankCombiner.class));
intermediateConfig.addChainedTask(SynchronousChainedCombineDriver.class, combinerConfig, "Combiner");
// ---------------- the tail (co group) --------------------
AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
degreeOfParallelism);
TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
tailConfig.setIterationId(ITERATION_ID);
tailConfig.setIsWorksetUpdate();
// inputs and driver
tailConfig.setDriver(CoGroupDriver.class);
tailConfig.setDriverStrategy(DriverStrategy.CO_GROUP);
tailConfig.addInputToGroup(0);
tailConfig.addInputToGroup(1);
tailConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
tailConfig.setInputSerializer(vertexWithRankSerializer, 1);
tailConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
tailConfig.setDriverComparator(vertexWithRankComparator, 1);
tailConfig.setDriverPairComparator(coGroupComparator);
tailConfig.setInputAsynchronouslyMaterialized(0, true);
tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
tailConfig.setInputComparator(vertexWithRankComparator, 1);
tailConfig.setRelativeMemoryInput(1, (double)coGroupSortMemory/totalMemoryConsumption);
tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
tailConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
// output
tailConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
// the stub
tailConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductCoGroup>(CustomCompensatableDotProductCoGroup.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", degreeOfParallelism);
TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
outputConfig.addInputToGroup(0);
outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
outputConfig.setStubWrapper(new UserCodeClassWrapper<CustomPageWithRankOutFormat>(CustomPageWithRankOutFormat.class));
outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath);
// --------------- the auxiliaries ---------------------
AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
syncConfig.setNumberOfIterations(numIterations);
syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
syncConfig.setConvergenceCriterion(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new DiffL1NormConvergenceCriterion());
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(head, sync, ChannelType.NETWORK, 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);