Skip to content

Commit

Permalink
Implemented dangling pagerank with generic data types.
Browse files Browse the repository at this point in the history
  • Loading branch information
sewen committed Feb 11, 2013
1 parent 5f58ce7 commit 3fe087d
Show file tree
Hide file tree
Showing 27 changed files with 1,003 additions and 146 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -129,10 +129,10 @@ public long element() {
}
}

public double elementAsDouble() {
String token = new String(buffer, tokenOffset, tokenNumBytes, Charsets.US_ASCII);
return Double.valueOf(token);
}
// public double elementAsDouble() {
// String token = new String(buffer, tokenOffset, tokenNumBytes, Charsets.US_ASCII);
// return Double.valueOf(token);
// }


@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception {
// --------------- the inputs ---------------------

// page rank input
JobInputVertex pageWithRankInput = JobGraphUtils.createInput(DanglingPageGenerateRankInputFormat.class,
JobInputVertex pageWithRankInput = JobGraphUtils.createInput(ImprovedDanglingPageRankInputFormat.class,
pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
Expand All @@ -120,7 +120,7 @@ public static JobGraph getJobGraph(String[] args) throws Exception {
pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));

// edges as adjacency list
JobInputVertex adjacencyListInput = JobGraphUtils.createInput(AdjacencyListInputFormat.class,
JobInputVertex adjacencyListInput = JobGraphUtils.createInput(ImprovedAdjacencyListInputFormat.class,
adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism, numSubTasksPerInstance);
TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
package eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom;

import eu.stratosphere.nephele.configuration.Configuration;
import eu.stratosphere.pact.common.stubs.Collector;
import eu.stratosphere.pact.generic.stub.AbstractStub;
import eu.stratosphere.pact.generic.stub.GenericCoGrouper;
import eu.stratosphere.pact.runtime.iterative.compensatable.ConfigUtils;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.DiffL1NormConvergenceCriterion;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.PageRankStats;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.PageRankStatsAggregator;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRank;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRankAndDangling;
import eu.stratosphere.pact.runtime.iterative.concurrent.IterationContext;

import java.util.Iterator;
import java.util.Set;

public class CustomCompensatableDotProductCoGroup extends AbstractStub implements GenericCoGrouper<VertexWithRankAndDangling, VertexWithRank, VertexWithRankAndDangling> {

private VertexWithRankAndDangling accumulator = new VertexWithRankAndDangling();

private PageRankStatsAggregator aggregator =
(PageRankStatsAggregator) new DiffL1NormConvergenceCriterion().createAggregator();

private long numVertices;

private long numDanglingVertices;

private double dampingFactor;

private double danglingRankFactor;

private static final double BETA = 0.85;

private int workerIndex;

private int currentIteration;

private int failingIteration;

private Set<Integer> failingWorkers;

@Override
public void open(Configuration parameters) throws Exception {
workerIndex = ConfigUtils.asInteger("pact.parallel.task.id", parameters);
currentIteration = ConfigUtils.asInteger("pact.iterations.currentIteration", parameters);
failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);

numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
numDanglingVertices = ConfigUtils.asLong("pageRank.numDanglingVertices", parameters);

aggregator.reset();

dampingFactor = (1d - BETA) / (double) numVertices;

if (currentIteration == 1) {
danglingRankFactor = BETA * (double) numDanglingVertices / ((double) numVertices * (double) numVertices);
} else {
PageRankStats previousAggregate = (PageRankStats) IterationContext.instance().getGlobalAggregate(
workerIndex);
danglingRankFactor = BETA * previousAggregate.danglingRank() / (double) numVertices;
}
}

@Override
public void coGroup(Iterator<VertexWithRankAndDangling> currentPageRankIterator, Iterator<VertexWithRank> partialRanks,
Collector<VertexWithRankAndDangling> collector)
{
if (!currentPageRankIterator.hasNext()) {
long missingVertex = partialRanks.next().getVertexID();
throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!");
}

VertexWithRankAndDangling currentPageRank = currentPageRankIterator.next();

long edges = 0;
double summedRank = 0;
while (partialRanks.hasNext()) {
summedRank += partialRanks.next().getRank();
edges++;
}

double rank = BETA * summedRank + dampingFactor + danglingRankFactor;

double currentRank = currentPageRank.getRank();
boolean isDangling = currentPageRank.isDangling();

double danglingRankToAggregate = isDangling ? rank : 0;
long danglingVerticesToAggregate = isDangling ? 1 : 0;

double diff = Math.abs(currentRank - rank);

aggregator.aggregate(diff, rank, danglingRankToAggregate, danglingVerticesToAggregate, 1, edges, summedRank, 0);

accumulator.setVertexID(currentPageRank.getVertexID());
accumulator.setRank(rank);
accumulator.setDangling(isDangling);

collector.collect(accumulator);
}

@Override
public void close() throws Exception {
if (currentIteration == failingIteration && failingWorkers.contains(workerIndex)) {
aggregator.reset();
}
IterationContext.instance().setAggregate(workerIndex, aggregator.getAggregate());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/***********************************************************************************************************************
*
* Copyright (C) 2012 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*
**********************************************************************************************************************/

package eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom;

import eu.stratosphere.nephele.configuration.Configuration;
import eu.stratosphere.pact.common.stubs.Collector;
import eu.stratosphere.pact.generic.stub.AbstractStub;
import eu.stratosphere.pact.generic.stub.GenericMatcher;
import eu.stratosphere.pact.runtime.iterative.compensatable.ConfigUtils;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithAdjacencyList;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRank;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRankAndDangling;

import java.util.Random;
import java.util.Set;

public class CustomCompensatableDotProductMatch extends AbstractStub implements
GenericMatcher<VertexWithRankAndDangling, VertexWithAdjacencyList, VertexWithRank> {

private VertexWithRank record = new VertexWithRank();

private Random random = new Random();

private double messageLoss;

private boolean isFailure;

@Override
public void open(Configuration parameters) throws Exception {
int workerIndex = ConfigUtils.asInteger("pact.parallel.task.id", parameters);
int currentIteration = ConfigUtils.asInteger("pact.iterations.currentIteration", parameters);
int failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
Set<Integer> failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
isFailure = currentIteration == failingIteration && failingWorkers.contains(workerIndex);
messageLoss = ConfigUtils.asDouble("compensation.messageLoss", parameters);
}

@Override
public void match(VertexWithRankAndDangling pageWithRank, VertexWithAdjacencyList adjacencyList, Collector<VertexWithRank> collector)
throws Exception
{
double rank = pageWithRank.getRank();
long[] adjacentNeighbors = adjacencyList.getTargets();
int numNeighbors = adjacencyList.getNumTargets();

double rankToDistribute = rank / (double) numNeighbors;
record.setRank(rankToDistribute);

for (int n = 0; n < numNeighbors; n++) {
record.setVertexID(adjacentNeighbors[n]);
if (isFailure) {
if (random.nextDouble() >= messageLoss) {
collector.collect(record);
}
} else {
collector.collect(record);
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
package eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom;

import eu.stratosphere.nephele.configuration.Configuration;
import eu.stratosphere.pact.common.stubs.Collector;
import eu.stratosphere.pact.generic.stub.AbstractStub;
import eu.stratosphere.pact.generic.stub.GenericMapper;
import eu.stratosphere.pact.runtime.iterative.compensatable.ConfigUtils;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.PageRankStats;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRankAndDangling;
import eu.stratosphere.pact.runtime.iterative.concurrent.IterationContext;

import java.util.Set;

public class CustomCompensatingMap extends AbstractStub implements GenericMapper<VertexWithRankAndDangling, VertexWithRankAndDangling> {

private boolean isFailureIteration;

private boolean isFailingWorker;

private double uniformRank;

private double rescaleFactor;

@Override
public void open(Configuration parameters) throws Exception {


int currentIteration = ConfigUtils.asInteger("pact.iterations.currentIteration", parameters);
int failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
isFailureIteration = currentIteration == failingIteration + 1;

int workerIndex = ConfigUtils.asInteger("pact.parallel.task.id", parameters);
Set<Integer> failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
isFailingWorker = failingWorkers.contains(workerIndex);

long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);

if (currentIteration > 1) {
PageRankStats stats = (PageRankStats) IterationContext.instance().getGlobalAggregate(workerIndex);

uniformRank = 1d / (double) numVertices;
double lostMassFactor = (numVertices - stats.numVertices()) / (double) numVertices;
rescaleFactor = (1 - lostMassFactor) / stats.rank();
}
}

@Override
public void map(VertexWithRankAndDangling pageWithRank, Collector<VertexWithRankAndDangling> out) throws Exception {

if (isFailureIteration) {
double rank = pageWithRank.getRank();

if (isFailingWorker) {
pageWithRank.setRank(uniformRank);
} else {
pageWithRank.setRank(rank * rescaleFactor);
}
}
out.collect(pageWithRank);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
package eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom;

import eu.stratosphere.pact.generic.io.DelimitedInputFormat;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.AsciiLongArrayView;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithAdjacencyList;

public class CustomImprovedAdjacencyListInputFormat extends DelimitedInputFormat<VertexWithAdjacencyList> {

private final AsciiLongArrayView arrayView = new AsciiLongArrayView();

@Override
public boolean readRecord(VertexWithAdjacencyList target, byte[] bytes, int offset, int numBytes) {

if (numBytes == 0) {
return false;
}

arrayView.set(bytes, offset, numBytes);

long[] list = target.getTargets();

try {

int pos = 0;
while (arrayView.next()) {

if (pos == 0) {
target.setVertexID(arrayView.element());
} else {
if (list.length <= pos - 1) {
list = new long[list.length < 16 ? 16 : list.length * 2];
target.setTargets(list);
}
list[pos - 1] = arrayView.element();
}
pos++;
}

target.setNumTargets(pos - 1);
} catch (RuntimeException e) {
throw new RuntimeException("Error parsing: " + arrayView.toString(), e);
}

return true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
package eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom;

import eu.stratosphere.nephele.configuration.Configuration;
import eu.stratosphere.pact.generic.io.DelimitedInputFormat;
import eu.stratosphere.pact.runtime.iterative.compensatable.ConfigUtils;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.AsciiLongArrayView;
import eu.stratosphere.pact.runtime.iterative.compensatable.danglingpagerank.custom.types.VertexWithRankAndDangling;

public class CustomImprovedDanglingPageRankInputFormat extends DelimitedInputFormat<VertexWithRankAndDangling> {

private AsciiLongArrayView arrayView = new AsciiLongArrayView();

private static final long DANGLING_MARKER = 1l;

private double initialRank;

@Override
public void configure(Configuration parameters) {
long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
initialRank = 1.0 / numVertices;
super.configure(parameters);
}

@Override
public boolean readRecord(VertexWithRankAndDangling target, byte[] bytes, int offset, int numBytes) {

arrayView.set(bytes, offset, numBytes);

try {
arrayView.next();
target.setVertexID(arrayView.element());

if (arrayView.next()) {
target.setDangling(arrayView.element() == DANGLING_MARKER);
} else {
target.setDangling(false);
}

} catch (NumberFormatException e) {
throw new RuntimeException("Error parsing " + arrayView.toString(), e);
}

target.setRank(initialRank);

return true;
}
}
Loading

0 comments on commit 3fe087d

Please sign in to comment.