You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ap...@apache.org on 2013/04/16 03:12:05 UTC
[1/2] GIRAPH-633: Rename VertexEdges to OutEdges (apresta)
Updated Branches:
refs/heads/trunk 38f1f1793 -> 158c0b118
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
index 6703b22..5d293eb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
@@ -25,7 +25,7 @@ import java.util.Random;
import java.util.Set;
import org.apache.giraph.bsp.BspInputSplit;
import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.io.VertexInputFormat;
import org.apache.giraph.io.VertexReader;
@@ -144,9 +144,9 @@ public class PseudoRandomVertexInputFormat extends
Random rand = new Random(vertexId);
DoubleWritable vertexValue = new DoubleWritable(rand.nextDouble());
// In order to save memory and avoid copying, we add directly to a
- // VertexEdges instance.
- VertexEdges<LongWritable, DoubleWritable> edges =
- getConf().createAndInitializeVertexEdges(edgesPerVertex);
+ // OutEdges instance.
+ OutEdges<LongWritable, DoubleWritable> edges =
+ getConf().createAndInitializeOutEdges(edgesPerVertex);
Set<LongWritable> destVertices = Sets.newHashSet();
for (long i = 0; i < edgesPerVertex; ++i) {
LongWritable destVertexId = new LongWritable();
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
index 54b6bbd..cc6b126 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
@@ -21,7 +21,7 @@ package org.apache.giraph.job;
import org.apache.giraph.combiner.Combiner;
import org.apache.giraph.conf.GiraphConstants;
import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.DefaultVertexResolver;
import org.apache.giraph.graph.DefaultVertexValueFactory;
import org.apache.giraph.graph.Vertex;
@@ -118,7 +118,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
vertexValueType = classList.get(VALUE_PARAM_INDEX);
edgeValueType = classList.get(EDGE_PARAM_INDEX);
messageValueType = classList.get(MSG_PARAM_INDEX);
- verifyVertexEdgesGenericTypes();
+ verifyOutEdgesGenericTypes();
verifyVertexInputFormatGenericTypes();
verifyEdgeInputFormatGenericTypes();
verifyVertexOutputFormatGenericTypes();
@@ -165,7 +165,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
VERTEX_RESOLVER_CLASS.getDefaultClass().getCanonicalName());
}
}
- if (conf.getVertexEdgesClass() == null) {
+ if (conf.getOutEdgesClass() == null) {
if (LOG.isInfoEnabled()) {
LOG.info("checkConfiguration: No class found for " +
VERTEX_EDGES_CLASS.getKey() + ", defaulting to " +
@@ -175,15 +175,15 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
}
/**
- * Verify matching generic types for a specific VertexEdges class.
+ * Verify matching generic types for a specific OutEdges class.
*
- * @param vertexEdgesClass {@link VertexEdges} class to check
+ * @param outEdgesClass {@link org.apache.giraph.edge.OutEdges} class to check
*/
- private void verifyVertexEdgesGenericTypesClass(
- Class<? extends VertexEdges<I, E>> vertexEdgesClass) {
+ private void verifyOutEdgesGenericTypesClass(
+ Class<? extends OutEdges<I, E>> outEdgesClass) {
List<Class<?>> classList = ReflectionUtils.getTypeArguments(
- VertexEdges.class, vertexEdgesClass);
- // VertexEdges implementations can be generic, in which case there are no
+ OutEdges.class, outEdgesClass);
+ // OutEdges implementations can be generic, in which case there are no
// types to check.
if (classList.isEmpty()) {
return;
@@ -205,14 +205,14 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
}
}
- /** Verify matching generic types in VertexEdges. */
- private void verifyVertexEdgesGenericTypes() {
- Class<? extends VertexEdges<I, E>> vertexEdgesClass =
- conf.getVertexEdgesClass();
- Class<? extends VertexEdges<I, E>> inputVertexEdgesClass =
- conf.getInputVertexEdgesClass();
- verifyVertexEdgesGenericTypesClass(vertexEdgesClass);
- verifyVertexEdgesGenericTypesClass(inputVertexEdgesClass);
+ /** Verify matching generic types in OutEdges. */
+ private void verifyOutEdgesGenericTypes() {
+ Class<? extends OutEdges<I, E>> outEdgesClass =
+ conf.getOutEdgesClass();
+ Class<? extends OutEdges<I, E>> inputOutEdgesClass =
+ conf.getInputOutEdgesClass();
+ verifyOutEdgesGenericTypesClass(outEdgesClass);
+ verifyOutEdgesGenericTypesClass(inputOutEdgesClass);
}
/** Verify matching generic types in VertexInputFormat. */
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
index 7c7fc2b..c4669d3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
@@ -73,7 +73,7 @@ public class ByteArrayPartition<I extends WritableComparable,
representativeVertex.initialize(
getConf().createVertexId(),
getConf().createVertexValue(),
- getConf().createVertexEdges());
+ getConf().createOutEdges());
useUnsafeSerialization = getConf().useUnsafeSerialization();
}
@@ -194,7 +194,7 @@ public class ByteArrayPartition<I extends WritableComparable,
representativeVertex.initialize(
getConf().createVertexId(),
getConf().createVertexValue(),
- getConf().createVertexEdges());
+ getConf().createOutEdges());
useUnsafeSerialization = getConf().useUnsafeSerialization();
for (int i = 0; i < size; ++i) {
progress();
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
index 53b9dd4..2d30bf9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
@@ -19,7 +19,7 @@
package org.apache.giraph.partition;
import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.Vertex;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
@@ -381,12 +381,12 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
* @throws IOException
*/
@SuppressWarnings("unchecked")
- private void writeVertexEdges(
+ private void writeOutEdges(
DataOutput output,
Vertex<I, V, E, M> vertex)
throws IOException {
vertex.getId().write(output);
- ((VertexEdges<I, E>) vertex.getEdges()).write(output);
+ ((OutEdges<I, E>) vertex.getEdges()).write(output);
}
/**
@@ -417,12 +417,12 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
* @param partition The partition owning the vertex
* @throws IOException
*/
- private void readVertexEdges(DataInput in, Partition<I, V, E, M> partition)
+ private void readOutEdges(DataInput in, Partition<I, V, E, M> partition)
throws IOException {
I id = conf.createVertexId();
id.readFields(in);
Vertex<I, V, E, M> v = partition.getVertex(id);
- VertexEdges<I, E> edges = conf.createVertexEdges();
+ OutEdges<I, E> edges = conf.createOutEdges();
edges.readFields(in);
v.setEdges(edges);
}
@@ -454,7 +454,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
inputStream = new DataInputStream(
new BufferedInputStream(new FileInputStream(file)));
for (int i = 0; i < numVertices; ++i) {
- readVertexEdges(inputStream, partition);
+ readOutEdges(inputStream, partition);
}
inputStream.close();
/*
@@ -501,7 +501,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
outputStream = new DataOutputStream(
new BufferedOutputStream(new FileOutputStream(file)));
for (Vertex<I, V, E, M> vertex : partition) {
- writeVertexEdges(outputStream, vertex);
+ writeOutEdges(outputStream, vertex);
}
outputStream.close();
}
@@ -530,7 +530,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
outputStream = new DataOutputStream(
new BufferedOutputStream(new FileOutputStream(file, true)));
for (Vertex<I, V, E, M> vertex : partition) {
- writeVertexEdges(outputStream, vertex);
+ writeOutEdges(outputStream, vertex);
}
outputStream.close();
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index c9a52d1..6016ba4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -30,6 +30,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.combiner.Combiner;
import org.apache.giraph.conf.GiraphConfiguration;
import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.VertexValueFactory;
import org.apache.giraph.io.EdgeInputFormat;
import org.apache.giraph.io.VertexInputFormat;
@@ -39,7 +40,6 @@ import org.apache.giraph.job.GiraphConfigurationValidator;
import org.apache.giraph.master.MasterCompute;
import org.apache.giraph.partition.Partition;
import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.edge.VertexEdges;
import org.apache.giraph.worker.WorkerContext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -91,7 +91,7 @@ public final class ConfigurationUtils {
OPTIONS.addOption("eip", "edgeInputPath", true, "Edge input path");
OPTIONS.addOption("op", "outputPath", true, "Vertex output path");
OPTIONS.addOption("c", "combiner", true, "Combiner class");
- OPTIONS.addOption("ve", "vertexEdges", true, "Vertex edges class");
+ OPTIONS.addOption("ve", "outEdges", true, "Vertex edges class");
OPTIONS.addOption("wc", "workerContext", true, "WorkerContext class");
OPTIONS.addOption("aw", "aggregatorWriter", true, "AggregatorWriter class");
OPTIONS.addOption("mc", "masterCompute", true, "MasterCompute class");
@@ -214,13 +214,13 @@ public final class ConfigurationUtils {
(Class<? extends Combiner>) Class.forName(cmd.getOptionValue("c")));
}
if (cmd.hasOption("ve")) {
- giraphConfiguration.setVertexEdgesClass(
- (Class<? extends VertexEdges>)
+ giraphConfiguration.setOutEdgesClass(
+ (Class<? extends OutEdges>)
Class.forName(cmd.getOptionValue("ve")));
}
if (cmd.hasOption("ive")) {
- giraphConfiguration.setInputVertexEdgesClass(
- (Class<? extends VertexEdges>)
+ giraphConfiguration.setInputOutEdgesClass(
+ (Class<? extends OutEdges>)
Class.forName(cmd.getOptionValue("ive")));
}
if (cmd.hasOption("wc")) {
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 5391aa0..c607ca3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -20,7 +20,7 @@ package org.apache.giraph.utils;
import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.zk.ZooKeeperExt;
import org.apache.giraph.zk.ZooKeeperExt.PathStat;
@@ -477,7 +477,7 @@ public class WritableUtils {
throws IOException {
vertex.getId().readFields(input);
vertex.getValue().readFields(input);
- ((VertexEdges<I, E>) vertex.getEdges()).readFields(input);
+ ((OutEdges<I, E>) vertex.getEdges()).readFields(input);
if (input.readBoolean()) {
vertex.voteToHalt();
} else {
@@ -506,7 +506,7 @@ public class WritableUtils {
Vertex<I, V, E, M> vertex = conf.createVertex();
I id = conf.createVertexId();
V value = conf.createVertexValue();
- VertexEdges<I, E> edges = conf.createVertexEdges();
+ OutEdges<I, E> edges = conf.createOutEdges();
vertex.initialize(id, value, edges);
reinitializeVertexFromDataInput(input, vertex, conf);
return vertex;
@@ -533,7 +533,7 @@ public class WritableUtils {
throws IOException {
vertex.getId().write(output);
vertex.getValue().write(output);
- ((VertexEdges<I, E>) vertex.getEdges()).write(output);
+ ((OutEdges<I, E>) vertex.getEdges()).write(output);
output.writeBoolean(vertex.isHalted());
}
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
index 4eeff01..2bb3292 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
@@ -27,15 +27,15 @@ import org.junit.Test;
import java.util.Collection;
import java.util.List;
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
import static org.junit.Assert.assertEquals;
/**
- * Tests {@link VertexEdges} implementations that allow parallel edges.
+ * Tests {@link OutEdges} implementations that allow parallel edges.
*/
public class TestMultiGraphEdges {
- /** {@link VertexEdges} classes to be tested. */
- private Collection<Class<? extends VertexEdges>> edgesClasses =
+ /** {@link OutEdges} classes to be tested. */
+ private Collection<Class<? extends OutEdges>> edgesClasses =
Lists.newArrayList();
@Before
@@ -47,20 +47,20 @@ public class TestMultiGraphEdges {
}
/**
- * Ensures that all multigraph {@link VertexEdges} implementations allow
+ * Ensures that all multigraph {@link OutEdges} implementations allow
* parallel edges.
*/
@Test
public void testParallelEdges() {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
testParallelEdgesClass(edgesClass);
}
}
private void testParallelEdgesClass(
- Class<? extends VertexEdges> edgesClass) {
- VertexEdges<LongWritable, DoubleWritable> edges =
- instantiateVertexEdges(edgesClass);
+ Class<? extends OutEdges> edgesClass) {
+ OutEdges<LongWritable, DoubleWritable> edges =
+ instantiateOutEdges(edgesClass);
// Initial edges list contains parallel edges.
List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
index b86f409..c300d11 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
@@ -28,15 +28,15 @@ import org.junit.Test;
import java.util.Collection;
import java.util.List;
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
import static org.junit.Assert.assertEquals;
/**
- * Tests {@link MultiRandomAccessVertexEdges} implementations.
+ * Tests {@link MultiRandomAccessOutEdges} implementations.
*/
public class TestMultiRandomAccessEdges {
- /** {@link MultiRandomAccessVertexEdges} classes to be tested. */
- private Collection<Class<? extends MultiRandomAccessVertexEdges>>
+ /** {@link MultiRandomAccessOutEdges} classes to be tested. */
+ private Collection<Class<? extends MultiRandomAccessOutEdges>>
edgesClasses = Lists.newArrayList();
@Before
@@ -45,22 +45,22 @@ public class TestMultiRandomAccessEdges {
}
/**
- * Ensures that all {@link MultiRandomAccessVertexEdges} implementations
+ * Ensures that all {@link MultiRandomAccessOutEdges} implementations
* correctly return edge values.
*/
@Test
public void testParallelEdges() {
- for (Class<? extends MultiRandomAccessVertexEdges> edgesClass :
+ for (Class<? extends MultiRandomAccessOutEdges> edgesClass :
edgesClasses) {
testParallelEdgesClass(edgesClass);
}
}
private void testParallelEdgesClass(
- Class<? extends MultiRandomAccessVertexEdges> edgesClass) {
- MultiRandomAccessVertexEdges<LongWritable, DoubleWritable> edges =
- (MultiRandomAccessVertexEdges<LongWritable, DoubleWritable>)
- instantiateVertexEdges(edgesClass);
+ Class<? extends MultiRandomAccessOutEdges> edgesClass) {
+ MultiRandomAccessOutEdges<LongWritable, DoubleWritable> edges =
+ (MultiRandomAccessOutEdges<LongWritable, DoubleWritable>)
+ instantiateOutEdges(edgesClass);
// Initial edges list contains parallel edges.
List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
index 79bf093..99a104d 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
@@ -28,15 +28,15 @@ import java.util.Collection;
import java.util.Iterator;
import java.util.List;
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
import static org.junit.Assert.assertEquals;
/**
- * Tests {@link VertexEdges} implementations with null edge values.
+ * Tests {@link OutEdges} implementations with null edge values.
*/
public class TestNullValueEdges {
- /** {@link VertexEdges} classes to be tested. */
- private Collection<Class<? extends MutableVertexEdges>>
+ /** {@link OutEdges} classes to be tested. */
+ private Collection<Class<? extends MutableOutEdges>>
edgesClasses = Lists.newArrayList();
@Before
@@ -47,16 +47,16 @@ public class TestNullValueEdges {
@Test
public void testEdges() {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
testEdgesClass(edgesClass);
}
}
private void testEdgesClass(
- Class<? extends VertexEdges> edgesClass) {
- VertexEdges<LongWritable, NullWritable> edges =
- (VertexEdges<LongWritable, NullWritable>)
- instantiateVertexEdges(edgesClass);
+ Class<? extends OutEdges> edgesClass) {
+ OutEdges<LongWritable, NullWritable> edges =
+ (OutEdges<LongWritable, NullWritable>)
+ instantiateOutEdges(edgesClass);
List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
EdgeFactory.create(new LongWritable(1)),
@@ -79,16 +79,16 @@ public class TestNullValueEdges {
*/
@Test
public void testMutateEdges() {
- for (Class<? extends MutableVertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends MutableOutEdges> edgesClass : edgesClasses) {
testMutateEdgesClass(edgesClass);
}
}
private void testMutateEdgesClass(
- Class<? extends MutableVertexEdges> edgesClass) {
- MutableVertexEdges<LongWritable, NullWritable> edges =
- (MutableVertexEdges<LongWritable, NullWritable>)
- instantiateVertexEdges(edgesClass);
+ Class<? extends MutableOutEdges> edgesClass) {
+ MutableOutEdges<LongWritable, NullWritable> edges =
+ (MutableOutEdges<LongWritable, NullWritable>)
+ instantiateOutEdges(edgesClass);
edges.initialize();
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
index 3c57281..a06de0a 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
@@ -27,15 +27,15 @@ import org.junit.Test;
import java.util.Collection;
import java.util.List;
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
import static org.junit.Assert.assertEquals;
/**
- * Tests {@link VertexEdges} implementations that disallow parallel edges.
+ * Tests {@link OutEdges} implementations that disallow parallel edges.
*/
public class TestStrictGraphEdges {
- /** {@link VertexEdges} classes to be tested. */
- private Collection<Class<? extends VertexEdges>> edgesClasses =
+ /** {@link OutEdges} classes to be tested. */
+ private Collection<Class<? extends OutEdges>> edgesClasses =
Lists.newArrayList();
@Before
@@ -45,20 +45,20 @@ public class TestStrictGraphEdges {
}
/**
- * Ensures that all strict graph {@link VertexEdges} implementations
+ * Ensures that all strict graph {@link OutEdges} implementations
* disallow parallel edges.
*/
@Test
public void testParallelEdges() {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
testParallelEdgesClass(edgesClass);
}
}
private void testParallelEdgesClass(
- Class<? extends VertexEdges> edgesClass) {
- VertexEdges<LongWritable, DoubleWritable> edges =
- instantiateVertexEdges(edgesClass);
+ Class<? extends OutEdges> edgesClass) {
+ OutEdges<LongWritable, DoubleWritable> edges =
+ instantiateOutEdges(edgesClass);
// Initial edges list contains parallel edges.
List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
index d3ad069..74d7054 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
@@ -27,16 +27,16 @@ import org.junit.Test;
import java.util.Collection;
import java.util.List;
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
/**
- * Tests {@link StrictRandomAccessVertexEdges} implementations.
+ * Tests {@link StrictRandomAccessOutEdges} implementations.
*/
public class TestStrictRandomAccessEdges {
- /** {@link StrictRandomAccessVertexEdges} classes to be tested. */
- private Collection<Class<? extends StrictRandomAccessVertexEdges>>
+ /** {@link StrictRandomAccessOutEdges} classes to be tested. */
+ private Collection<Class<? extends StrictRandomAccessOutEdges>>
edgesClasses = Lists.newArrayList();
@Before
@@ -46,22 +46,22 @@ public class TestStrictRandomAccessEdges {
}
/**
- * Ensures that all {@link StrictRandomAccessVertexEdges} implementations
+ * Ensures that all {@link StrictRandomAccessOutEdges} implementations
* correctly return edge values.
*/
@Test
public void testParallelEdges() {
- for (Class<? extends StrictRandomAccessVertexEdges> edgesClass :
+ for (Class<? extends StrictRandomAccessOutEdges> edgesClass :
edgesClasses) {
testParallelEdgesClass(edgesClass);
}
}
private void testParallelEdgesClass(
- Class<? extends StrictRandomAccessVertexEdges> edgesClass) {
- StrictRandomAccessVertexEdges<LongWritable, DoubleWritable> edges =
- (StrictRandomAccessVertexEdges<LongWritable, DoubleWritable>)
- instantiateVertexEdges(edgesClass);
+ Class<? extends StrictRandomAccessOutEdges> edgesClass) {
+ StrictRandomAccessOutEdges<LongWritable, DoubleWritable> edges =
+ (StrictRandomAccessOutEdges<LongWritable, DoubleWritable>)
+ instantiateOutEdges(edgesClass);
// Initial edges list contains parallel edges.
List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
index 8a048fd..15d2bb0 100644
--- a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
@@ -29,7 +29,7 @@ import org.apache.giraph.edge.HashMultimapEdges;
import org.apache.giraph.edge.LongDoubleArrayEdges;
import org.apache.giraph.edge.LongDoubleHashMapEdges;
import org.apache.giraph.edge.MutableEdge;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.time.SystemTime;
import org.apache.giraph.time.Time;
import org.apache.giraph.time.Times;
@@ -57,14 +57,14 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/**
- * Test {@link Vertex} functionality across the provided {@link VertexEdges}
+ * Test {@link Vertex} functionality across the provided {@link org.apache.giraph.edge.OutEdges}
* classes.
*/
public class TestVertexAndEdges {
/** Number of repetitions. */
public static final int REPS = 100;
- /** {@link org.apache.giraph.edge.VertexEdges} classes to be tested. */
- private Collection<Class<? extends VertexEdges>> edgesClasses =
+ /** {@link org.apache.giraph.edge.OutEdges} classes to be tested. */
+ private Collection<Class<? extends OutEdges>> edgesClasses =
Lists.newArrayList();
/**
@@ -77,12 +77,12 @@ public class TestVertexAndEdges {
}
/**
- * A basic {@link VertexEdges} implementation that doesn't provide any
+ * A basic {@link org.apache.giraph.edge.OutEdges} implementation that doesn't provide any
* special functionality. Used to test the default implementations of
* Vertex#getEdgeValue(), Vertex#getMutableEdges(), etc.
*/
- public static class TestVertexEdges
- implements VertexEdges<LongWritable, DoubleWritable> {
+ public static class TestOutEdges
+ implements OutEdges<LongWritable, DoubleWritable> {
private List<Edge<LongWritable, DoubleWritable>> edgeList;
@@ -151,7 +151,7 @@ public class TestVertexAndEdges {
@Before
public void setUp() {
- edgesClasses.add(TestVertexEdges.class);
+ edgesClasses.add(TestOutEdges.class);
edgesClasses.add(ByteArrayEdges.class);
edgesClasses.add(ArrayListEdges.class);
edgesClasses.add(HashMapEdges.class);
@@ -161,10 +161,10 @@ public class TestVertexAndEdges {
}
private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
- instantiateVertex(Class<? extends VertexEdges> edgesClass) {
+ instantiateVertex(Class<? extends OutEdges> edgesClass) {
GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
giraphConfiguration.setVertexClass(TestVertex.class);
- giraphConfiguration.setVertexEdgesClass(edgesClass);
+ giraphConfiguration.setOutEdgesClass(edgesClass);
ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
new ImmutableClassesGiraphConfiguration(giraphConfiguration);
return immutableClassesGiraphConfiguration.createVertex();
@@ -185,34 +185,34 @@ public class TestVertexAndEdges {
assertEquals(5.5f, vertex.getValue().get(), 0d);
}
- public static VertexEdges
- instantiateVertexEdges(Class<? extends VertexEdges> edgesClass) {
+ public static OutEdges
+ instantiateOutEdges(Class<? extends OutEdges> edgesClass) {
GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
// Needed to extract type arguments in ReflectionUtils.
giraphConfiguration.setVertexClass(TestVertex.class);
- giraphConfiguration.setVertexEdgesClass(edgesClass);
+ giraphConfiguration.setOutEdgesClass(edgesClass);
ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
new ImmutableClassesGiraphConfiguration(giraphConfiguration);
- return immutableClassesGiraphConfiguration.createVertexEdges();
+ return immutableClassesGiraphConfiguration.createOutEdges();
}
/**
- * Test the provided {@link VertexEdges} implementations for instantiation,
+ * Test the provided {@link org.apache.giraph.edge.OutEdges} implementations for instantiation,
* initialization, edge addition, and edge removal.
*/
@Test
public void testEdges() {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
testEdgesClass(edgesClass);
}
}
- private void testEdgesClass(Class<? extends VertexEdges> edgesClass) {
+ private void testEdgesClass(Class<? extends OutEdges> edgesClass) {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
instantiateVertex(edgesClass);
- VertexEdges<LongWritable, DoubleWritable> vertexEdges =
- instantiateVertexEdges(edgesClass);
- assertNotNull(vertexEdges);
+ OutEdges<LongWritable, DoubleWritable> outEdges =
+ instantiateOutEdges(edgesClass);
+ assertNotNull(outEdges);
List<Edge<LongWritable, DoubleWritable>> edges = Lists.newLinkedList();
for (int i = 1000; i > 0; --i) {
@@ -220,8 +220,8 @@ public class TestVertexAndEdges {
new DoubleWritable(i * 2.0)));
}
- vertexEdges.initialize(edges);
- vertex.initialize(new LongWritable(1), new FloatWritable(1), vertexEdges);
+ outEdges.initialize(edges);
+ vertex.initialize(new LongWritable(1), new FloatWritable(1), outEdges);
assertEquals(20.0, vertex.getEdgeValue(new LongWritable(10)).get(), 0.0);
@@ -246,19 +246,19 @@ public class TestVertexAndEdges {
*/
@Test
public void testMutateEdges() {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
testMutateEdgesClass(edgesClass);
}
}
- private void testMutateEdgesClass(Class<? extends VertexEdges> edgesClass) {
+ private void testMutateEdgesClass(Class<? extends OutEdges> edgesClass) {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
instantiateVertex(edgesClass);
- VertexEdges<LongWritable, DoubleWritable> vertexEdges =
- instantiateVertexEdges(edgesClass);
+ OutEdges<LongWritable, DoubleWritable> outEdges =
+ instantiateOutEdges(edgesClass);
- vertexEdges.initialize();
- vertex.initialize(new LongWritable(0), new FloatWritable(0), vertexEdges);
+ outEdges.initialize();
+ vertex.initialize(new LongWritable(0), new FloatWritable(0), outEdges);
// Add 10 edges with id i, value i for i = 0..9
for (int i = 0; i < 10; ++i) {
@@ -327,24 +327,24 @@ public class TestVertexAndEdges {
}
/**
- * Test {@link Vertex} and {@link VertexEdges} serialization.
+ * Test {@link Vertex} and {@link org.apache.giraph.edge.OutEdges} serialization.
* @throws IOException
*/
@Test
public void testSerialize() throws IOException {
- for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
- testSerializeVertexEdgesClass(edgesClass);
- testDynamicChannelBufferSerializeVertexEdgesClass(edgesClass);
- testUnsafeSerializeVertexEdgesClass(edgesClass);
+ for (Class<? extends OutEdges> edgesClass : edgesClasses) {
+ testSerializeOutEdgesClass(edgesClass);
+ testDynamicChannelBufferSerializeOutEdgesClass(edgesClass);
+ testUnsafeSerializeOutEdgesClass(edgesClass);
}
}
private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
- buildVertex(Class<? extends VertexEdges> edgesClass) {
+ buildVertex(Class<? extends OutEdges> edgesClass) {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
instantiateVertex(edgesClass);
- VertexEdges<LongWritable, DoubleWritable> vertexEdges =
- instantiateVertexEdges(edgesClass);
+ OutEdges<LongWritable, DoubleWritable> outEdges =
+ instantiateOutEdges(edgesClass);
int edgesCount = 200;
List<Edge<LongWritable, DoubleWritable>> edges =
@@ -354,14 +354,14 @@ public class TestVertexAndEdges {
new DoubleWritable(i * 2.0)));
}
- vertexEdges.initialize(edges);
+ outEdges.initialize(edges);
vertex.initialize(new LongWritable(2), new FloatWritable(3.0f),
- vertexEdges);
+ outEdges);
return vertex;
}
- private void testSerializeVertexEdgesClass(
- Class<? extends VertexEdges> edgesClass) {
+ private void testSerializeOutEdgesClass(
+ Class<? extends OutEdges> edgesClass) {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
buildVertex(edgesClass);
@@ -404,8 +404,8 @@ public class TestVertexAndEdges {
assertTrue(EdgeIterables.sameEdges(vertex.getEdges(), readVertex.getEdges()));
}
- private void testDynamicChannelBufferSerializeVertexEdgesClass(
- Class<? extends VertexEdges> edgesClass)
+ private void testDynamicChannelBufferSerializeOutEdgesClass(
+ Class<? extends OutEdges> edgesClass)
throws IOException {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
buildVertex(edgesClass);
@@ -423,7 +423,7 @@ public class TestVertexAndEdges {
serializeNanosStart);
}
serializeNanos /= REPS;
- System.out.println("testDynamicChannelBufferSerializeVertexEdgesClass: " +
+ System.out.println("testDynamicChannelBufferSerializeOutEdgesClass: " +
"Serializing took " + serializeNanos + " ns for " +
outputStream.getDynamicChannelBuffer().writerIndex() + " bytes " +
(outputStream.getDynamicChannelBuffer().writerIndex() * 1f *
@@ -447,7 +447,7 @@ public class TestVertexAndEdges {
outputStream.getDynamicChannelBuffer().readerIndex(0);
}
deserializeNanos /= REPS;
- System.out.println("testDynamicChannelBufferSerializeVertexEdgesClass: " +
+ System.out.println("testDynamicChannelBufferSerializeOutEdgesClass: " +
"Deserializing took " + deserializeNanos + " ns for " +
outputStream.getDynamicChannelBuffer().writerIndex() + " bytes " +
(outputStream.getDynamicChannelBuffer().writerIndex() * 1f *
@@ -459,8 +459,8 @@ public class TestVertexAndEdges {
assertTrue(EdgeIterables.sameEdges(vertex.getEdges(), readVertex.getEdges()));
}
- private void testUnsafeSerializeVertexEdgesClass(
- Class<? extends VertexEdges> edgesClass)
+ private void testUnsafeSerializeOutEdgesClass(
+ Class<? extends OutEdges> edgesClass)
throws IOException {
Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
buildVertex(edgesClass);
@@ -478,7 +478,7 @@ public class TestVertexAndEdges {
serializeNanosStart);
}
serializeNanos /= REPS;
- System.out.println("testUnsafeSerializeVertexEdgesClass: " +
+ System.out.println("testUnsafeSerializeOutEdgesClass: " +
"Serializing took " +
serializeNanos +
" ns for " + outputStream.getPos()
@@ -503,7 +503,7 @@ public class TestVertexAndEdges {
deserializeNanosStart);
}
deserializeNanos /= REPS;
- System.out.println("testUnsafeSerializeVertexEdgesClass: " +
+ System.out.println("testUnsafeSerializeOutEdgesClass: " +
"Deserializing took " +
deserializeNanos +
" ns for " + outputStream.getPos() +
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
index ede76f0..07d4cc8 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
@@ -65,7 +65,7 @@ public class TestEdgeInput extends BspCase {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(TestVertexWithNumEdges.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -94,7 +94,7 @@ public class TestEdgeInput extends BspCase {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(TestVertexWithNumEdges.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setEdgeInputFormatClass(IntNullReverseTextEdgeInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -130,7 +130,7 @@ public class TestEdgeInput extends BspCase {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(TestVertexDoNothing.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -161,7 +161,7 @@ public class TestEdgeInput extends BspCase {
conf = new GiraphConfiguration();
conf.setVertexClass(TestVertexWithNumEdges.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -179,7 +179,7 @@ public class TestEdgeInput extends BspCase {
assertEquals(1, (int) values.get(5));
}
- // It should use the specified input VertexEdges class.
+ // It should use the specified input OutEdges class.
@Test
public void testDifferentInputEdgesClass() throws Exception {
String[] edges = new String[] {
@@ -191,8 +191,8 @@ public class TestEdgeInput extends BspCase {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(TestVertexCheckEdgesType.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
- conf.setInputVertexEdgesClass(TestVertexEdgesFilterEven.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
+ conf.setInputOutEdgesClass(TestOutEdgesFilterEven.class);
conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -221,7 +221,7 @@ public class TestEdgeInput extends BspCase {
public static class TestVertexCheckEdgesType extends TestVertexWithNumEdges {
@Override
public void compute(Iterable<NullWritable> messages) throws IOException {
- assertFalse(getEdges() instanceof TestVertexEdgesFilterEven);
+ assertFalse(getEdges() instanceof TestOutEdgesFilterEven);
assertTrue(getEdges() instanceof ByteArrayEdges);
super.compute(messages);
}
@@ -247,7 +247,7 @@ public class TestEdgeInput extends BspCase {
}
}
- public static class TestVertexEdgesFilterEven
+ public static class TestOutEdgesFilterEven
extends ByteArrayEdges<IntWritable, NullWritable> {
@Override
public void add(Edge<IntWritable, NullWritable> edge) {
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java b/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
index 1c34f78..9d6b215 100644
--- a/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
+++ b/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
@@ -89,7 +89,7 @@ public class TestMasterObserver {
conf.set(GiraphConstants.MASTER_OBSERVER_CLASSES.getKey(),
arrayToString(klasses));
conf.setVertexClass(NoOpVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(IntNullNullTextInputFormat.class);
InternalVertexRunner.run(conf, graph);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
index bdcc49e..9b655af 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
@@ -19,7 +19,7 @@ package org.apache.giraph.partition;
import org.apache.giraph.edge.ArrayListEdges;
import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.GiraphTransferRegulator;
import org.apache.giraph.job.GiraphJob;
import org.apache.giraph.graph.Vertex;
@@ -71,7 +71,7 @@ public class TestGiraphTransferRegulator {
.setInt(GiraphTransferRegulator.MAX_VERTICES_PER_TRANSFER, 1);
job.getConfiguration()
.setInt(GiraphTransferRegulator.MAX_EDGES_PER_TRANSFER, 3);
- VertexEdges<IntWritable, DoubleWritable> edges =
+ OutEdges<IntWritable, DoubleWritable> edges =
new ArrayListEdges<IntWritable, DoubleWritable>();
edges.initialize(3);
edges.add(EdgeFactory.create(new IntWritable(2), new DoubleWritable(22)));
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
index 5d3fae1..5d71259 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
@@ -67,7 +67,7 @@ public class ConnectedComponentsVertexTest {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(ConnectedComponentsVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setCombinerClass(MinimumIntCombiner.class);
conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
index f748cb0..e4c029c 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
@@ -58,7 +58,7 @@ public class ConnectedComponentsVertexTestInMemory {
public void testToyData() throws Exception {
GiraphConfiguration conf = new GiraphConfiguration();
conf.setVertexClass(ConnectedComponentsVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setCombinerClass(MinimumIntCombiner.class);
TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> graph =
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
index b41bcf6..2a39291 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
@@ -51,7 +51,7 @@ public class PageRankVertexTest {
conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 50);
conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
conf.setVertexClass(PageRankVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(LongDoubleNullTextInputFormat.class);
conf.setVertexOutputFormatClass(
VertexWithDoubleValueNullEdgeTextOutputFormat.class);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
index 6ecfefe..a2a891e 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
@@ -45,7 +45,7 @@ public class RandomWalkWithRestartVertexTest {
conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.25f);
conf.setVertexClass(RandomWalkWithRestartVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(LongDoubleDoubleTextInputFormat.class);
conf.setVertexOutputFormatClass(
VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
@@ -81,7 +81,7 @@ public class RandomWalkWithRestartVertexTest {
conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
conf.setVertexClass(RandomWalkWithRestartVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(
NormalizingLongDoubleDoubleTextInputFormat.class);
conf.setVertexOutputFormatClass(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
index ee99dc5..1006577 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
@@ -124,7 +124,7 @@ public class SimpleShortestPathsVertexTest {
// start from vertex 1
SOURCE_ID.set(conf, 1);
conf.setVertexClass(SimpleShortestPathsVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setVertexInputFormatClass(
JsonLongDoubleFloatDoubleVertexInputFormat.class);
conf.setVertexOutputFormatClass(
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
index 4945b52..23d055e 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
@@ -71,7 +71,7 @@ public class TryMultiIpcBindingPortsTest {
GiraphConfiguration conf = new GiraphConfiguration();
GiraphConstants.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT.set(conf, true);
conf.setVertexClass(ConnectedComponentsVertex.class);
- conf.setVertexEdgesClass(ByteArrayEdges.class);
+ conf.setOutEdgesClass(ByteArrayEdges.class);
conf.setCombinerClass(MinimumIntCombiner.class);
conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/src/site/xdoc/io.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/io.xml b/src/site/xdoc/io.xml
index cb752cb..76f610b 100644
--- a/src/site/xdoc/io.xml
+++ b/src/site/xdoc/io.xml
@@ -1,23 +1,22 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements. See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership. The ASF licenses this file
-to you 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.
--->
+ ~ Licensed to the Apache Software Foundation (ASF) under one
+ ~ or more contributor license agreements. See the NOTICE file
+ ~ distributed with this work for additional information
+ ~ regarding copyright ownership. The ASF licenses this file
+ ~ to you 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.
+ -->
<document xmlns="http://maven.apache.org/XDOC/2.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
[2/2] git commit: updated refs/heads/trunk to 158c0b1
Posted by ap...@apache.org.
GIRAPH-633: Rename VertexEdges to OutEdges (apresta)
Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/158c0b11
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/158c0b11
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/158c0b11
Branch: refs/heads/trunk
Commit: 158c0b1180fa8f01bffd119df49b982fdda22771
Parents: 38f1f17
Author: Alessandro Presta <al...@fb.com>
Authored: Mon Apr 15 11:40:47 2013 -0700
Committer: Alessandro Presta <al...@fb.com>
Committed: Mon Apr 15 13:53:47 2013 -0700
----------------------------------------------------------------------
.../apache/giraph/benchmark/PageRankBenchmark.java | 2 +-
.../giraph/benchmark/ShortestPathsBenchmark.java | 6 +-
.../benchmark/WeightedPageRankBenchmark.java | 16 ++--
.../java/org/apache/giraph/conf/GiraphClasses.java | 48 ++++----
.../apache/giraph/conf/GiraphConfiguration.java | 34 +++---
.../org/apache/giraph/conf/GiraphConstants.java | 14 +-
.../conf/ImmutableClassesGiraphConfiguration.java | 95 ++++++++-------
.../org/apache/giraph/edge/ArrayListEdges.java | 6 +-
.../org/apache/giraph/edge/ByteArrayEdges.java | 6 +-
.../apache/giraph/edge/ConfigurableOutEdges.java | 36 ++++++
.../giraph/edge/ConfigurableVertexEdges.java | 36 ------
.../java/org/apache/giraph/edge/EdgeStore.java | 52 ++++----
.../java/org/apache/giraph/edge/HashMapEdges.java | 8 +-
.../org/apache/giraph/edge/HashMultimapEdges.java | 6 +-
.../org/apache/giraph/edge/IntNullArrayEdges.java | 4 +-
.../apache/giraph/edge/LongDoubleArrayEdges.java | 6 +-
.../apache/giraph/edge/LongDoubleHashMapEdges.java | 8 +-
.../org/apache/giraph/edge/LongNullArrayEdges.java | 6 +-
.../apache/giraph/edge/LongNullHashSetEdges.java | 6 +-
.../giraph/edge/MultiRandomAccessOutEdges.java | 41 ++++++
.../giraph/edge/MultiRandomAccessVertexEdges.java | 41 ------
.../apache/giraph/edge/MutableEdgesIterable.java | 6 +-
.../apache/giraph/edge/MutableEdgesWrapper.java | 24 ++--
.../org/apache/giraph/edge/MutableOutEdges.java | 42 +++++++
.../org/apache/giraph/edge/MutableVertexEdges.java | 42 -------
.../main/java/org/apache/giraph/edge/OutEdges.java | 84 +++++++++++++
.../apache/giraph/edge/ReuseObjectsOutEdges.java | 34 +++++
.../giraph/edge/ReuseObjectsVertexEdges.java | 34 -----
.../giraph/edge/StrictRandomAccessOutEdges.java | 51 ++++++++
.../giraph/edge/StrictRandomAccessVertexEdges.java | 51 --------
.../java/org/apache/giraph/edge/VertexEdges.java | 84 -------------
.../main/java/org/apache/giraph/graph/Vertex.java | 55 ++++----
.../io/formats/PseudoRandomEdgeInputFormat.java | 12 +-
.../PseudoRandomIntNullVertexInputFormat.java | 6 +-
.../io/formats/PseudoRandomVertexInputFormat.java | 8 +-
.../giraph/job/GiraphConfigurationValidator.java | 34 +++---
.../giraph/partition/ByteArrayPartition.java | 4 +-
.../giraph/partition/DiskBackedPartitionStore.java | 16 ++--
.../apache/giraph/utils/ConfigurationUtils.java | 12 +-
.../org/apache/giraph/utils/WritableUtils.java | 8 +-
.../apache/giraph/edge/TestMultiGraphEdges.java | 18 ++--
.../giraph/edge/TestMultiRandomAccessEdges.java | 20 ++--
.../org/apache/giraph/edge/TestNullValueEdges.java | 28 ++--
.../apache/giraph/edge/TestStrictGraphEdges.java | 18 ++--
.../giraph/edge/TestStrictRandomAccessEdges.java | 20 ++--
.../apache/giraph/graph/TestVertexAndEdges.java | 96 +++++++-------
.../java/org/apache/giraph/io/TestEdgeInput.java | 18 ++--
.../apache/giraph/master/TestMasterObserver.java | 2 +-
.../partition/TestGiraphTransferRegulator.java | 4 +-
.../examples/ConnectedComponentsVertexTest.java | 2 +-
.../ConnectedComponentsVertexTestInMemory.java | 2 +-
.../apache/giraph/examples/PageRankVertexTest.java | 2 +-
.../examples/RandomWalkWithRestartVertexTest.java | 4 +-
.../examples/SimpleShortestPathsVertexTest.java | 2 +-
.../examples/TryMultiIpcBindingPortsTest.java | 2 +-
src/site/xdoc/io.xml | 33 +++---
56 files changed, 679 insertions(+), 676 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
index 3a755c8..0f8d284 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
@@ -45,7 +45,7 @@ public class PageRankBenchmark extends GiraphBenchmark {
protected void prepareConfiguration(GiraphConfiguration conf,
CommandLine cmd) {
conf.setVertexClass(PageRankVertex.class);
- conf.setVertexEdgesClass(IntNullArrayEdges.class);
+ conf.setOutEdgesClass(IntNullArrayEdges.class);
conf.setCombinerClass(FloatSumCombiner.class);
conf.setVertexInputFormatClass(
PseudoRandomIntNullVertexInputFormat.class);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
index df53ee6..8e6c877 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
@@ -41,7 +41,7 @@ public class ShortestPathsBenchmark extends GiraphBenchmark {
private static final Logger LOG =
Logger.getLogger(ShortestPathsBenchmark.class);
- /** Option for VertexEdges class */
+ /** Option for OutEdges class */
private static final BenchmarkOption EDGES_CLASS = new BenchmarkOption(
"c", "edgesClass", true,
"Vertex edges class (0 for HashMapEdges, 1 for ArrayListEdges)");
@@ -60,9 +60,9 @@ public class ShortestPathsBenchmark extends GiraphBenchmark {
CommandLine cmd) {
conf.setVertexClass(ShortestPathsVertex.class);
if (EDGES_CLASS.getOptionIntValue(cmd, 1) == 1) {
- conf.setVertexEdgesClass(ArrayListEdges.class);
+ conf.setOutEdgesClass(ArrayListEdges.class);
} else {
- conf.setVertexEdgesClass(HashMapEdges.class);
+ conf.setOutEdgesClass(HashMapEdges.class);
}
LOG.info("Using class " + GiraphConstants.VERTEX_CLASS.get(conf));
conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
index 9de10b5..3fc514a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
@@ -45,7 +45,7 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
private static final Logger LOG =
Logger.getLogger(WeightedPageRankBenchmark.class);
- /** Option for VertexEdges class */
+ /** Option for OutEdges class */
private static final BenchmarkOption EDGES_CLASS = new BenchmarkOption(
"c", "edgesClass", true,
"Vertex edges class (0 for LongDoubleArrayEdges," +
@@ -92,25 +92,25 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
int edgesClassOption = EDGES_CLASS.getOptionIntValue(cmd, 1);
switch (edgesClassOption) {
case 0:
- configuration.setVertexEdgesClass(LongDoubleArrayEdges.class);
+ configuration.setOutEdgesClass(LongDoubleArrayEdges.class);
break;
case 1:
- configuration.setVertexEdgesClass(ByteArrayEdges.class);
+ configuration.setOutEdgesClass(ByteArrayEdges.class);
break;
case 2:
- configuration.setVertexEdgesClass(ByteArrayEdges.class);
+ configuration.setOutEdgesClass(ByteArrayEdges.class);
configuration.useUnsafeSerialization(true);
break;
case 3:
- configuration.setVertexEdgesClass(ArrayListEdges.class);
+ configuration.setOutEdgesClass(ArrayListEdges.class);
break;
case 4:
- configuration.setVertexEdgesClass(HashMapEdges.class);
+ configuration.setOutEdgesClass(HashMapEdges.class);
break;
default:
- LOG.info("Unknown VertexEdges class, " +
+ LOG.info("Unknown OutEdges class, " +
"defaulting to LongDoubleArrayEdges");
- configuration.setVertexEdgesClass(LongDoubleArrayEdges.class);
+ configuration.setOutEdgesClass(LongDoubleArrayEdges.class);
}
LOG.info("Using edges class " +
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
index 95499bd..4a0e8f7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
@@ -21,7 +21,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.aggregators.TextAggregatorWriter;
import org.apache.giraph.combiner.Combiner;
import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.DefaultVertexResolver;
import org.apache.giraph.graph.DefaultVertexValueFactory;
import org.apache.giraph.graph.Vertex;
@@ -70,9 +70,9 @@ public class GiraphClasses<I extends WritableComparable,
/** Message value class - cached for fast access */
protected Class<M> messageValueClass;
/** Vertex edges class - cached for fast access */
- protected Class<? extends VertexEdges<I, E>> vertexEdgesClass;
+ protected Class<? extends OutEdges<I, E>> outEdgesClass;
/** Input vertex edges class - cached for fast access */
- protected Class<? extends VertexEdges<I, E>> inputVertexEdgesClass;
+ protected Class<? extends OutEdges<I, E>> inputOutEdgesClass;
/** Vertex value factory class - cached for fast access */
protected Class<? extends VertexValueFactory<V>> vertexValueFactoryClass;
@@ -114,9 +114,9 @@ public class GiraphClasses<I extends WritableComparable,
public GiraphClasses() {
// Note: the cast to Object is required in order for javac to accept the
// downcast.
- vertexEdgesClass = (Class<? extends VertexEdges<I, E>>) (Object)
+ outEdgesClass = (Class<? extends OutEdges<I, E>>) (Object)
ByteArrayEdges.class;
- inputVertexEdgesClass = (Class<? extends VertexEdges<I, E>>) (Object)
+ inputOutEdgesClass = (Class<? extends OutEdges<I, E>>) (Object)
ByteArrayEdges.class;
vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>) (Object)
DefaultVertexValueFactory.class;
@@ -157,10 +157,10 @@ public class GiraphClasses<I extends WritableComparable,
edgeValueClass = (Class<E>) classList.get(2);
messageValueClass = (Class<M>) classList.get(3);
- vertexEdgesClass = (Class<? extends VertexEdges<I, E>>)
+ outEdgesClass = (Class<? extends OutEdges<I, E>>)
VERTEX_EDGES_CLASS.get(conf);
- inputVertexEdgesClass = (Class<? extends VertexEdges<I, E>>)
- INPUT_VERTEX_EDGES_CLASS.getWithDefault(conf, vertexEdgesClass);
+ inputOutEdgesClass = (Class<? extends OutEdges<I, E>>)
+ INPUT_VERTEX_EDGES_CLASS.getWithDefault(conf, outEdgesClass);
vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>)
VERTEX_VALUE_FACTORY_CLASS.get(conf);
@@ -237,16 +237,16 @@ public class GiraphClasses<I extends WritableComparable,
*
* @return Vertex edges class.
*/
- public Class<? extends VertexEdges<I, E>> getVertexEdgesClass() {
- return vertexEdgesClass;
+ public Class<? extends OutEdges<I, E>> getOutEdgesClass() {
+ return outEdgesClass;
}
/* Get Vertex edges class used during edge-based input
*
* @return Vertex edges class.
*/
- public Class<? extends VertexEdges<I, E>> getInputVertexEdgesClass() {
- return inputVertexEdgesClass;
+ public Class<? extends OutEdges<I, E>> getInputOutEdgesClass() {
+ return inputOutEdgesClass;
}
@@ -508,29 +508,29 @@ public class GiraphClasses<I extends WritableComparable,
}
/**
- * Set VertexEdges class held
+ * Set OutEdges class held
*
- * @param vertexEdgesClass Vertex edges class to set
+ * @param outEdgesClass Vertex edges class to set
* @return this
*/
- public GiraphClasses setVertexEdgesClass(
- Class<? extends VertexEdges> vertexEdgesClass) {
- this.vertexEdgesClass =
- (Class<? extends VertexEdges<I, E>>) vertexEdgesClass;
+ public GiraphClasses setOutEdgesClass(
+ Class<? extends OutEdges> outEdgesClass) {
+ this.outEdgesClass =
+ (Class<? extends OutEdges<I, E>>) outEdgesClass;
return this;
}
/**
- * Set VertexEdges class used during edge-input (if different from the one
+ * Set OutEdges class used during edge-input (if different from the one
* used for computation)
*
- * @param inputVertexEdgesClass Input vertex edges class to set
+ * @param inputOutEdgesClass Input vertex edges class to set
* @return this
*/
- public GiraphClasses setInputVertexEdgesClass(
- Class<? extends VertexEdges> inputVertexEdgesClass) {
- this.inputVertexEdgesClass =
- (Class<? extends VertexEdges<I, E>>) inputVertexEdgesClass;
+ public GiraphClasses setInputOutEdgesClass(
+ Class<? extends OutEdges> inputOutEdgesClass) {
+ this.inputOutEdgesClass =
+ (Class<? extends OutEdges<I, E>>) inputOutEdgesClass;
return this;
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
index c527595..7f9e38e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
@@ -20,8 +20,8 @@ package org.apache.giraph.conf;
import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.combiner.Combiner;
-import org.apache.giraph.edge.ReuseObjectsVertexEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.edge.ReuseObjectsOutEdges;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.graph.VertexResolver;
import org.apache.giraph.graph.VertexValueFactory;
@@ -100,36 +100,36 @@ public class GiraphConfiguration extends Configuration
*
* @return vertex edges class
*/
- public Class<? extends VertexEdges> getVertexEdgesClass() {
+ public Class<? extends OutEdges> getOutEdgesClass() {
return VERTEX_EDGES_CLASS.get(this);
}
/**
* Set the vertex edges class
*
- * @param vertexEdgesClass Determines the way edges are stored
+ * @param outEdgesClass Determines the way edges are stored
*/
- public final void setVertexEdgesClass(
- Class<? extends VertexEdges> vertexEdgesClass) {
- VERTEX_EDGES_CLASS.set(this, vertexEdgesClass);
+ public final void setOutEdgesClass(
+ Class<? extends OutEdges> outEdgesClass) {
+ VERTEX_EDGES_CLASS.set(this, outEdgesClass);
}
/**
* Set the vertex edges class used during edge-based input (if different
* from the one used during computation)
*
- * @param inputVertexEdgesClass Determines the way edges are stored
+ * @param inputOutEdgesClass Determines the way edges are stored
*/
- public final void setInputVertexEdgesClass(
- Class<? extends VertexEdges> inputVertexEdgesClass) {
- INPUT_VERTEX_EDGES_CLASS.set(this, inputVertexEdgesClass);
+ public final void setInputOutEdgesClass(
+ Class<? extends OutEdges> inputOutEdgesClass) {
+ INPUT_VERTEX_EDGES_CLASS.set(this, inputOutEdgesClass);
}
/**
- * True if the {@link VertexEdges} implementation copies the passed edges
- * to its own data structure, i.e. it doesn't keep references to Edge
- * objects, target vertex ids or edge values passed to add() or
- * initialize().
+ * True if the {@link org.apache.giraph.edge.OutEdges} implementation
+ * copies the passed edges to its own data structure,
+ * i.e. it doesn't keep references to Edge objects, target vertex ids or edge
+ * values passed to add() or initialize().
* This makes it possible to reuse edge objects passed to the data
* structure, to minimize object instantiation (see for example
* EdgeStore#addPartitionEdges()).
@@ -137,8 +137,8 @@ public class GiraphConfiguration extends Configuration
* @return True iff we can reuse the edge objects
*/
public boolean reuseEdgeObjects() {
- return ReuseObjectsVertexEdges.class.isAssignableFrom(
- getVertexEdgesClass());
+ return ReuseObjectsOutEdges.class.isAssignableFrom(
+ getOutEdgesClass());
}
/**
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
index a55a2a2..6a5949e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
@@ -21,7 +21,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.aggregators.TextAggregatorWriter;
import org.apache.giraph.combiner.Combiner;
import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.graph.DefaultVertexResolver;
import org.apache.giraph.graph.DefaultVertexValueFactory;
import org.apache.giraph.graph.Vertex;
@@ -66,13 +66,13 @@ public interface GiraphConstants {
ClassConfOption.create("giraph.vertexValueFactoryClass",
DefaultVertexValueFactory.class, VertexValueFactory.class);
/** Vertex edges class - optional */
- ClassConfOption<VertexEdges> VERTEX_EDGES_CLASS =
- ClassConfOption.create("giraph.vertexEdgesClass", ByteArrayEdges.class,
- VertexEdges.class);
+ ClassConfOption<OutEdges> VERTEX_EDGES_CLASS =
+ ClassConfOption.create("giraph.outEdgesClass", ByteArrayEdges.class,
+ OutEdges.class);
/** Vertex edges class to be used during edge input only - optional */
- ClassConfOption<VertexEdges> INPUT_VERTEX_EDGES_CLASS =
- ClassConfOption.create("giraph.inputVertexEdgesClass",
- ByteArrayEdges.class, VertexEdges.class);
+ ClassConfOption<OutEdges> INPUT_VERTEX_EDGES_CLASS =
+ ClassConfOption.create("giraph.inputOutEdgesClass",
+ ByteArrayEdges.class, OutEdges.class);
/** Class for Master - optional */
ClassConfOption<MasterCompute> MASTER_COMPUTE_CLASS =
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
index dc3a26f..8dfe546 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -22,8 +22,8 @@ import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.combiner.Combiner;
import org.apache.giraph.edge.Edge;
import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.edge.ReusableEdge;
-import org.apache.giraph.edge.VertexEdges;
import org.apache.giraph.graph.GraphState;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.graph.VertexResolver;
@@ -588,96 +588,99 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
}
@Override
- public Class<? extends VertexEdges<I, E>> getVertexEdgesClass() {
- return classes.getVertexEdgesClass();
+ public Class<? extends OutEdges<I, E>> getOutEdgesClass() {
+ return classes.getOutEdgesClass();
}
/**
- * Get the user's subclassed {@link VertexEdges} used for input
+ * Get the user's subclassed {@link org.apache.giraph.edge.OutEdges} used for
+ * input
*
* @return User's input vertex edges class
*/
- public Class<? extends VertexEdges<I, E>> getInputVertexEdgesClass() {
- return classes.getInputVertexEdgesClass();
+ public Class<? extends OutEdges<I, E>> getInputOutEdgesClass() {
+ return classes.getInputOutEdgesClass();
}
/**
- * Check whether the user has specified a different {@link VertexEdges}
- * class to be used during edge-based input.
+ * Check whether the user has specified a different
+ * {@link org.apache.giraph.edge.OutEdges} class to be used during
+ * edge-based input.
*
* @return True iff there is a special edges class for input
*/
- public boolean useInputVertexEdges() {
- return classes.getInputVertexEdgesClass() != classes.getVertexEdgesClass();
+ public boolean useInputOutEdges() {
+ return classes.getInputOutEdgesClass() != classes.getOutEdgesClass();
}
/**
- * Create a user {@link VertexEdges}
+ * Create a user {@link org.apache.giraph.edge.OutEdges}
*
- * @return Instantiated user VertexEdges
+ * @return Instantiated user OutEdges
*/
- public VertexEdges<I, E> createVertexEdges() {
- return ReflectionUtils.newInstance(getVertexEdgesClass(), this);
+ public OutEdges<I, E> createOutEdges() {
+ return ReflectionUtils.newInstance(getOutEdgesClass(), this);
}
/**
- * Create a {@link VertexEdges} instance and initialize it with the default
- * capacity.
+ * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+ * it with the default capacity.
*
- * @return Instantiated VertexEdges
+ * @return Instantiated OutEdges
*/
- public VertexEdges<I, E> createAndInitializeVertexEdges() {
- VertexEdges<I, E> vertexEdges = createVertexEdges();
- vertexEdges.initialize();
- return vertexEdges;
+ public OutEdges<I, E> createAndInitializeOutEdges() {
+ OutEdges<I, E> outEdges = createOutEdges();
+ outEdges.initialize();
+ return outEdges;
}
/**
- * Create a {@link VertexEdges} instance and initialize it with the given
- * capacity (the number of edges that will be added).
+ * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+ * it with the given capacity (the number of edges that will be added).
*
* @param capacity Number of edges that will be added
- * @return Instantiated VertexEdges
+ * @return Instantiated OutEdges
*/
- public VertexEdges<I, E> createAndInitializeVertexEdges(int capacity) {
- VertexEdges<I, E> vertexEdges = createVertexEdges();
- vertexEdges.initialize(capacity);
- return vertexEdges;
+ public OutEdges<I, E> createAndInitializeOutEdges(int capacity) {
+ OutEdges<I, E> outEdges = createOutEdges();
+ outEdges.initialize(capacity);
+ return outEdges;
}
/**
- * Create a {@link VertexEdges} instance and initialize it with the given
- * iterable of edges.
+ * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+ * it with the given iterable of edges.
*
* @param edges Iterable of edges to add
- * @return Instantiated VertexEdges
+ * @return Instantiated OutEdges
*/
- public VertexEdges<I, E> createAndInitializeVertexEdges(
+ public OutEdges<I, E> createAndInitializeOutEdges(
Iterable<Edge<I, E>> edges) {
- VertexEdges<I, E> vertexEdges = createVertexEdges();
- vertexEdges.initialize(edges);
- return vertexEdges;
+ OutEdges<I, E> outEdges = createOutEdges();
+ outEdges.initialize(edges);
+ return outEdges;
}
/**
- * Create a user {@link VertexEdges} used during edge-based input
+ * Create a user {@link org.apache.giraph.edge.OutEdges} used during
+ * edge-based input
*
- * @return Instantiated user input VertexEdges
+ * @return Instantiated user input OutEdges
*/
- public VertexEdges<I, E> createInputVertexEdges() {
- return ReflectionUtils.newInstance(getInputVertexEdgesClass(), this);
+ public OutEdges<I, E> createInputOutEdges() {
+ return ReflectionUtils.newInstance(getInputOutEdgesClass(), this);
}
/**
- * Create an input {@link VertexEdges} instance and initialize it with the
- * default capacity.
+ * Create an input {@link org.apache.giraph.edge.OutEdges} instance and
+ * initialize it with the default capacity.
*
- * @return Instantiated input VertexEdges
+ * @return Instantiated input OutEdges
*/
- public VertexEdges<I, E> createAndInitializeInputVertexEdges() {
- VertexEdges<I, E> vertexEdges = createInputVertexEdges();
- vertexEdges.initialize();
- return vertexEdges;
+ public OutEdges<I, E> createAndInitializeInputOutEdges() {
+ OutEdges<I, E> outEdges = createInputOutEdges();
+ outEdges.initialize();
+ return outEdges;
}
/**
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
index dda7568..4eb3378 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
@@ -30,15 +30,15 @@ import java.util.ArrayList;
import java.util.Iterator;
/**
- * {@link VertexEdges} implementation backed by an {@link ArrayList}.
+ * {@link OutEdges} implementation backed by an {@link ArrayList}.
* Parallel edges are allowed.
*
* @param <I> Vertex id
* @param <E> Edge value
*/
public class ArrayListEdges<I extends WritableComparable, E extends Writable>
- extends ConfigurableVertexEdges<I, E>
- implements MutableVertexEdges<I, E> {
+ extends ConfigurableOutEdges<I, E>
+ implements MutableOutEdges<I, E> {
/** List of edges. */
private ArrayList<Edge<I, E>> edgeList;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
index 16748ef..b9d1f6c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
@@ -33,7 +33,7 @@ import java.util.LinkedList;
import java.util.List;
/**
- * {@link VertexEdges} implementation backed by a byte array.
+ * {@link OutEdges} implementation backed by a byte array.
* Parallel edges are allowed.
* Note: this implementation is optimized for space usage,
* but edge removals are expensive.
@@ -42,8 +42,8 @@ import java.util.List;
* @param <E> Edge value
*/
public class ByteArrayEdges<I extends WritableComparable, E extends Writable>
- extends ConfigurableVertexEdges<I, E>
- implements ReuseObjectsVertexEdges<I, E> {
+ extends ConfigurableOutEdges<I, E>
+ implements ReuseObjectsOutEdges<I, E> {
/** Serialized edges. */
private byte[] serializedEdges;
/** Number of bytes used in serializedEdges. */
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
new file mode 100644
index 0000000..631b209
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Common base class for {@link OutEdges} implementations that require a
+ * configuration.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public abstract class ConfigurableOutEdges<I extends WritableComparable,
+ E extends Writable>
+ extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E,
+ Writable> implements OutEdges<I, E> {
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
deleted file mode 100644
index f05e0d7..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Common base class for {@link VertexEdges} implementations that require a
- * configuration.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public abstract class ConfigurableVertexEdges<I extends WritableComparable,
- E extends Writable>
- extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E,
- Writable> implements VertexEdges<I, E> {
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
index e8cb620..81044bd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
@@ -55,17 +55,17 @@ public class EdgeStore<I extends WritableComparable,
private Progressable progressable;
/** Map used to temporarily store incoming edges. */
private ConcurrentMap<Integer,
- ConcurrentMap<I, VertexEdges<I, E>>> transientEdges;
+ ConcurrentMap<I, OutEdges<I, E>>> transientEdges;
/**
- * Whether the chosen {@link VertexEdges} implementation allows for Edge
+ * Whether the chosen {@link OutEdges} implementation allows for Edge
* reuse.
*/
private boolean reuseEdgeObjects;
/**
- * Whether the {@link VertexEdges} class used during input is different
+ * Whether the {@link OutEdges} class used during input is different
* from the one used during computation.
*/
- private boolean useInputVertexEdges;
+ private boolean useInputOutEdges;
/**
* Constructor.
@@ -84,7 +84,7 @@ public class EdgeStore<I extends WritableComparable,
transientEdges = new MapMaker().concurrencyLevel(
configuration.getNettyServerExecutionConcurrency()).makeMap();
reuseEdgeObjects = configuration.reuseEdgeObjects();
- useInputVertexEdges = configuration.useInputVertexEdges();
+ useInputOutEdges = configuration.useInputOutEdges();
}
/**
@@ -96,10 +96,10 @@ public class EdgeStore<I extends WritableComparable,
*/
public void addPartitionEdges(
int partitionId, ByteArrayVertexIdEdges<I, E> edges) {
- ConcurrentMap<I, VertexEdges<I, E>> partitionEdges =
+ ConcurrentMap<I, OutEdges<I, E>> partitionEdges =
transientEdges.get(partitionId);
if (partitionEdges == null) {
- ConcurrentMap<I, VertexEdges<I, E>> newPartitionEdges =
+ ConcurrentMap<I, OutEdges<I, E>> newPartitionEdges =
new MapMaker().concurrencyLevel(
configuration.getNettyServerExecutionConcurrency()).makeMap();
partitionEdges = transientEdges.putIfAbsent(partitionId,
@@ -116,38 +116,38 @@ public class EdgeStore<I extends WritableComparable,
Edge<I, E> edge = reuseEdgeObjects ?
vertexIdEdgeIterator.getCurrentEdge() :
vertexIdEdgeIterator.releaseCurrentEdge();
- VertexEdges<I, E> vertexEdges = partitionEdges.get(vertexId);
- if (vertexEdges == null) {
- VertexEdges<I, E> newVertexEdges =
- configuration.createAndInitializeInputVertexEdges();
- vertexEdges = partitionEdges.putIfAbsent(vertexId, newVertexEdges);
- if (vertexEdges == null) {
- vertexEdges = newVertexEdges;
+ OutEdges<I, E> outEdges = partitionEdges.get(vertexId);
+ if (outEdges == null) {
+ OutEdges<I, E> newOutEdges =
+ configuration.createAndInitializeInputOutEdges();
+ outEdges = partitionEdges.putIfAbsent(vertexId, newOutEdges);
+ if (outEdges == null) {
+ outEdges = newOutEdges;
// Since we had to use the vertex id as a new key in the map,
// we need to release the object.
vertexIdEdgeIterator.releaseCurrentVertexId();
}
}
- synchronized (vertexEdges) {
- vertexEdges.add(edge);
+ synchronized (outEdges) {
+ outEdges.add(edge);
}
}
}
/**
- * Convert the input edges to the {@link VertexEdges} data structure used
+ * Convert the input edges to the {@link OutEdges} data structure used
* for computation (if different).
*
* @param inputEdges Input edges
* @return Compute edges
*/
- private VertexEdges<I, E> convertInputToComputeEdges(
- VertexEdges<I, E> inputEdges) {
- if (!useInputVertexEdges) {
+ private OutEdges<I, E> convertInputToComputeEdges(
+ OutEdges<I, E> inputEdges) {
+ if (!useInputOutEdges) {
return inputEdges;
} else {
- VertexEdges<I, E> computeEdges =
- configuration.createAndInitializeVertexEdges(inputEdges.size());
+ OutEdges<I, E> computeEdges =
+ configuration.createAndInitializeOutEdges(inputEdges.size());
for (Edge<I, E> edge : inputEdges) {
computeEdges.add(edge);
}
@@ -186,10 +186,10 @@ public class EdgeStore<I extends WritableComparable,
while ((partitionId = partitionIdQueue.poll()) != null) {
Partition<I, V, E, M> partition =
service.getPartitionStore().getPartition(partitionId);
- ConcurrentMap<I, VertexEdges<I, E>> partitionEdges =
+ ConcurrentMap<I, OutEdges<I, E>> partitionEdges =
transientEdges.remove(partitionId);
for (I vertexId : partitionEdges.keySet()) {
- VertexEdges<I, E> vertexEdges = convertInputToComputeEdges(
+ OutEdges<I, E> outEdges = convertInputToComputeEdges(
partitionEdges.remove(vertexId));
Vertex<I, V, E, M> vertex = partition.getVertex(vertexId);
// If the source vertex doesn't exist, create it. Otherwise,
@@ -197,10 +197,10 @@ public class EdgeStore<I extends WritableComparable,
if (vertex == null) {
vertex = configuration.createVertex();
vertex.initialize(vertexId, configuration.createVertexValue(),
- vertexEdges);
+ outEdges);
partition.putVertex(vertex);
} else {
- vertex.setEdges(vertexEdges);
+ vertex.setEdges(outEdges);
// Some Partition implementations (e.g. ByteArrayPartition)
// require us to put back the vertex after modifying it.
partition.saveVertex(vertex);
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
index 9fa7b64..fc23cf0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
@@ -31,7 +31,7 @@ import java.util.Iterator;
import java.util.Map;
/**
- * {@link VertexEdges} implementation backed by a {@link HashMap}.
+ * {@link OutEdges} implementation backed by a {@link HashMap}.
* Parallel edges are not allowed.
* Note: this implementation is optimized for fast random access and mutations,
* but uses more space.
@@ -40,9 +40,9 @@ import java.util.Map;
* @param <E> Edge value
*/
public class HashMapEdges<I extends WritableComparable, E extends Writable>
- extends ConfigurableVertexEdges<I, E>
- implements StrictRandomAccessVertexEdges<I, E>,
- MutableVertexEdges<I, E> {
+ extends ConfigurableOutEdges<I, E>
+ implements StrictRandomAccessOutEdges<I, E>,
+ MutableOutEdges<I, E> {
/** Map from target vertex id to edge value. */
private HashMap<I, E> edgeMap;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
index 123d49f..abe6151 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
@@ -31,7 +31,7 @@ import java.util.Iterator;
import java.util.Map;
/**
- * {@link VertexEdges} implementation backed by an {@link ArrayListMultimap}.
+ * {@link OutEdges} implementation backed by an {@link ArrayListMultimap}.
* Parallel edges are allowed.
* Note: this implementation is optimized for fast mutations,
* but uses more space.
@@ -40,8 +40,8 @@ import java.util.Map;
* @param <E> Edge value
*/
public class HashMultimapEdges<I extends WritableComparable, E extends Writable>
- extends ConfigurableVertexEdges<I, E>
- implements MultiRandomAccessVertexEdges<I, E> {
+ extends ConfigurableOutEdges<I, E>
+ implements MultiRandomAccessOutEdges<I, E> {
/** Multimap from target vertex id to edge values. */
private ArrayListMultimap<I, E> edgeMultimap;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
index 2363caf..9867771 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
@@ -33,14 +33,14 @@ import java.util.Collection;
import java.util.Iterator;
/**
- * Implementation of {@link VertexEdges} with int ids and null edge
+ * Implementation of {@link OutEdges} with int ids and null edge
* values, backed by dynamic primitive array.
* Parallel edges are allowed.
* Note: this implementation is optimized for space usage,
* but edge removals are expensive.
*/
public class IntNullArrayEdges
- implements ReuseObjectsVertexEdges<IntWritable, NullWritable> {
+ implements ReuseObjectsOutEdges<IntWritable, NullWritable> {
/** Array of target vertex ids */
private IntArrayList neighbors;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
index 6ce85d5..45d182e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
@@ -33,15 +33,15 @@ import java.util.Collection;
import java.util.Iterator;
/**
- * Implementation of {@link VertexEdges} with long ids and double edge
+ * Implementation of {@link OutEdges} with long ids and double edge
* values, backed by dynamic primitive arrays.
* Parallel edges are allowed.
* Note: this implementation is optimized for space usage,
* but edge removals are expensive.
*/
public class LongDoubleArrayEdges
- implements ReuseObjectsVertexEdges<LongWritable, DoubleWritable>,
- MutableVertexEdges<LongWritable, DoubleWritable> {
+ implements ReuseObjectsOutEdges<LongWritable, DoubleWritable>,
+ MutableOutEdges<LongWritable, DoubleWritable> {
/** Array of target vertex ids. */
private LongArrayList neighbors;
/** Array of edge values. */
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
index 908ad25..c33200d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
@@ -32,7 +32,7 @@ import java.util.Collection;
import java.util.Iterator;
/**
- * {@link VertexEdges} implementation with long ids and double edge values,
+ * {@link OutEdges} implementation with long ids and double edge values,
* backed by a {@link Long2DoubleOpenHashMap}.
* Parallel edges are not allowed.
* Note: this implementation is optimized for fast random access and mutations,
@@ -40,9 +40,9 @@ import java.util.Iterator;
* {@link LongDoubleArrayEdges}.
*/
public class LongDoubleHashMapEdges
- implements StrictRandomAccessVertexEdges<LongWritable, DoubleWritable>,
- ReuseObjectsVertexEdges<LongWritable, DoubleWritable>,
- MutableVertexEdges<LongWritable, DoubleWritable> {
+ implements StrictRandomAccessOutEdges<LongWritable, DoubleWritable>,
+ ReuseObjectsOutEdges<LongWritable, DoubleWritable>,
+ MutableOutEdges<LongWritable, DoubleWritable> {
/** Hash map from target vertex id to edge value. */
private Long2DoubleOpenHashMap edgeMap;
/** Representative edge value object, used by getEdgeValue(). */
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
index fee27f1..d753588 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
@@ -30,15 +30,15 @@ import java.util.Collection;
import java.util.Iterator;
/**
- * Implementation of {@link VertexEdges} with long ids and null edge
+ * Implementation of {@link OutEdges} with long ids and null edge
* values, backed by a dynamic primitive array.
* Parallel edges are allowed.
* Note: this implementation is optimized for space usage,
* but random access and edge removals are expensive.
*/
public class LongNullArrayEdges
- implements ReuseObjectsVertexEdges<LongWritable, NullWritable>,
- MutableVertexEdges<LongWritable, NullWritable> {
+ implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+ MutableOutEdges<LongWritable, NullWritable> {
/** Array of target vertex ids. */
private LongArrayList neighbors;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
index f374e36..ef0302c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
@@ -30,7 +30,7 @@ import java.util.Collection;
import java.util.Iterator;
/**
- * {@link VertexEdges} implementation with long ids and null edge values,
+ * {@link OutEdges} implementation with long ids and null edge values,
* backed by a {@link LongOpenHashSet}.
* Parallel edges are not allowed.
* Note: this implementation is optimized for fast random access and mutations,
@@ -38,8 +38,8 @@ import java.util.Iterator;
* {@link LongNullArrayEdges}.
*/
public class LongNullHashSetEdges
- implements ReuseObjectsVertexEdges<LongWritable, NullWritable>,
- MutableVertexEdges<LongWritable, NullWritable> {
+ implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+ MutableOutEdges<LongWritable, NullWritable> {
/** Hash set of target vertex ids. */
private LongOpenHashSet neighbors;
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
new file mode 100644
index 0000000..e3fc16b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for {@link OutEdges} implementations that provide efficient
+ * random access to the edges given the target vertex id.
+ * This version is for multigraphs (i.e. there can be parallel edges).
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface MultiRandomAccessOutEdges<I extends WritableComparable,
+ E extends Writable> extends OutEdges<I, E> {
+ /**
+ * Return an iterable over the edge values for a given target vertex id.
+ *
+ * @param targetVertexId Target vertex id
+ * @return Iterable of edge values
+ */
+ Iterable<E> getAllEdgeValues(I targetVertexId);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
deleted file mode 100644
index 9f8658e..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for {@link VertexEdges} implementations that provide efficient
- * random access to the edges given the target vertex id.
- * This version is for multigraphs (i.e. there can be parallel edges).
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface MultiRandomAccessVertexEdges<I extends WritableComparable,
- E extends Writable> extends VertexEdges<I, E> {
- /**
- * Return an iterable over the edge values for a given target vertex id.
- *
- * @param targetVertexId Target vertex id
- * @return Iterable of edge values
- */
- Iterable<E> getAllEdgeValues(I targetVertexId);
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
index 9dab09c..82486f4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
@@ -26,7 +26,7 @@ import java.util.Iterator;
/**
* Helper class to provide a mutable iterable over the edges when the chosen
- * {@link VertexEdges} doesn't offer a specialized one.
+ * {@link OutEdges} doesn't offer a specialized one.
*
* @param <I> Vertex id
* @param <E> Edge value
@@ -48,7 +48,7 @@ public class MutableEdgesIterable<I extends WritableComparable,
@Override
public Iterator<MutableEdge<I, E>> iterator() {
final MutableEdgesWrapper<I, E> mutableEdgesWrapper =
- MutableEdgesWrapper.wrap((VertexEdges<I, E>) vertex.getEdges(),
+ MutableEdgesWrapper.wrap((OutEdges<I, E>) vertex.getEdges(),
vertex.getConf());
vertex.setEdges(mutableEdgesWrapper);
@@ -57,7 +57,7 @@ public class MutableEdgesIterable<I extends WritableComparable,
private Iterator<Edge<I, E>> oldEdgesIterator =
mutableEdgesWrapper.getOldEdgesIterator();
/** New edges data structure. */
- private VertexEdges<I, E> newEdges = mutableEdgesWrapper.getNewEdges();
+ private OutEdges<I, E> newEdges = mutableEdgesWrapper.getNewEdges();
@Override
public boolean hasNext() {
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
index 5b20bf7..cd845d0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
@@ -31,8 +31,8 @@ import java.util.Iterator;
* Helper class that wraps the current out-edges and inserts them into a new
* data structure as they are iterated over.
* Used by Vertex to provide a mutable iterator when the chosen
- * {@link VertexEdges} doesn't offer a specialized one.
- * The edges are "unwrapped" back to the chosen {@link VertexEdges} data
+ * {@link OutEdges} doesn't offer a specialized one.
+ * The edges are "unwrapped" back to the chosen {@link OutEdges} data
* structure as soon as possible: either when the iterator is exhausted,
* or after compute() if iteration has been terminated early.
*
@@ -40,9 +40,9 @@ import java.util.Iterator;
* @param <E> Edge value
*/
public class MutableEdgesWrapper<I extends WritableComparable,
- E extends Writable> implements VertexEdges<I, E> {
+ E extends Writable> implements OutEdges<I, E> {
/** New edges data structure (initially empty). */
- private final VertexEdges<I, E> newEdges;
+ private final OutEdges<I, E> newEdges;
/** Iterator over the old edges. */
private final Iterator<Edge<I, E>> oldEdgesIterator;
/** Last edge that was returned during iteration. */
@@ -55,8 +55,8 @@ public class MutableEdgesWrapper<I extends WritableComparable,
* @param oldEdges Current out-edges
* @param newEdges New (empty) edges data structure
*/
- private MutableEdgesWrapper(VertexEdges<I, E> oldEdges,
- VertexEdges<I, E> newEdges) {
+ private MutableEdgesWrapper(OutEdges<I, E> oldEdges,
+ OutEdges<I, E> newEdges) {
oldEdgesIterator = oldEdges.iterator();
this.newEdges = newEdges;
}
@@ -72,19 +72,19 @@ public class MutableEdgesWrapper<I extends WritableComparable,
*/
public static <I extends WritableComparable, E extends Writable>
MutableEdgesWrapper<I, E> wrap(
- VertexEdges<I, E> edges,
+ OutEdges<I, E> edges,
ImmutableClassesGiraphConfiguration<I, ?, E, ?> conf) {
MutableEdgesWrapper<I, E> wrapper = new MutableEdgesWrapper<I, E>(
- edges, conf.createAndInitializeVertexEdges(edges.size()));
+ edges, conf.createAndInitializeOutEdges(edges.size()));
return wrapper;
}
/**
* Moves all the remaining edges to the new data structure, and returns it.
*
- * @return The new {@link VertexEdges} data structure.
+ * @return The new {@link OutEdges} data structure.
*/
- public VertexEdges<I, E> unwrap() {
+ public OutEdges<I, E> unwrap() {
if (currentEdge != null) {
newEdges.add(currentEdge);
currentEdge = null;
@@ -96,11 +96,11 @@ public class MutableEdgesWrapper<I extends WritableComparable,
}
/**
- * Get the new {@link VertexEdges} data structure.
+ * Get the new {@link OutEdges} data structure.
*
* @return New edges
*/
- public VertexEdges<I, E> getNewEdges() {
+ public OutEdges<I, E> getNewEdges() {
return newEdges;
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
new file mode 100644
index 0000000..0dcc018
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.util.Iterator;
+
+/**
+ * Interface for {@link OutEdges} implementations that have an optimized
+ * mutable edge iterator.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface MutableOutEdges<I extends WritableComparable,
+ E extends Writable> extends OutEdges<I, E> {
+ /**
+ * Returns an iterator over edges that can be modified in-place,
+ * either by changing the current edge value or by removing the current edge.
+ *
+ * @return A mutable edge iterator
+ */
+ Iterator<MutableEdge<I, E>> mutableIterator();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
deleted file mode 100644
index 503de92..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-import java.util.Iterator;
-
-/**
- * Interface for {@link VertexEdges} implementations that have an optimized
- * mutable edge iterator.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface MutableVertexEdges<I extends WritableComparable,
- E extends Writable> extends VertexEdges<I, E> {
- /**
- * Returns an iterator over edges that can be modified in-place,
- * either by changing the current edge value or by removing the current edge.
- *
- * @return A mutable edge iterator
- */
- Iterator<MutableEdge<I, E>> mutableIterator();
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
new file mode 100644
index 0000000..34dc5cb
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for data structures that store out-edges for a vertex.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface OutEdges<I extends WritableComparable, E extends Writable>
+ extends Iterable<Edge<I, E>>, Writable {
+ /**
+ * Initialize the data structure and set the edges from an iterable.
+ * This method (or one of the two alternatives) must be called
+ * after instantiation, unless readFields() is called.
+ * Note: whether parallel edges are allowed or not depends on the
+ * implementation.
+ *
+ * @param edges Iterable of edges
+ */
+ void initialize(Iterable<Edge<I, E>> edges);
+
+ /**
+ * Initialize the data structure with the specified initial capacity.
+ * This method (or one of the two alternatives) must be called
+ * after instantiation, unless readFields() is called.
+ *
+ * @param capacity Initial capacity
+ */
+ void initialize(int capacity);
+
+ /**
+ * Initialize the data structure with the default initial capacity.
+ * This method (or one of the two alternatives) must be called
+ * after instantiation, unless readFields() is called.
+ *
+ */
+ void initialize();
+
+ /**
+ * Add an edge.
+ * Note: whether parallel edges are allowed or not depends on the
+ * implementation.
+ *
+ * @param edge Edge to add
+ */
+ void add(Edge<I, E> edge);
+
+ /**
+ * Remove all edges to the given target vertex.
+ * Note: the implementation will vary depending on whether parallel edges
+ * are allowed or not.
+ *
+ * @param targetVertexId Target vertex id
+ */
+ void remove(I targetVertexId);
+
+ /**
+ * Return the number of edges.
+ *
+ * @return Number of edges
+ */
+ int size();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
new file mode 100644
index 0000000..27cf9ca
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Empty interface to characterize {@link OutEdges} implementations that
+ * don't keep references to the Edge (or id and value) objects they are passed.
+ * The Giraph infrastructure can exploit this characteristic by reusing Edge
+ * objects.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface ReuseObjectsOutEdges<I extends WritableComparable,
+ E extends Writable> extends OutEdges<I, E> { }
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
deleted file mode 100644
index 7704baf..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Empty interface to characterize {@link VertexEdges} implementations that
- * don't keep references to the Edge (or id and value) objects they are passed.
- * The Giraph infrastructure can exploit this characteristic by reusing Edge
- * objects.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface ReuseObjectsVertexEdges<I extends WritableComparable,
- E extends Writable> extends VertexEdges<I, E> { }
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
new file mode 100644
index 0000000..800f45b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for {@link OutEdges} implementations that provide efficient
+ * random access to the edges given the target vertex id.
+ * This version is for strict graphs (i.e. assumes no parallel edges).
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface StrictRandomAccessOutEdges<I extends WritableComparable,
+ E extends Writable> extends OutEdges<I, E> {
+ /**
+ * Return the edge value for the given target vertex id (or null if there
+ * is no edge pointing to it).
+ *
+ * @param targetVertexId Target vertex id
+ * @return Edge value
+ */
+ E getEdgeValue(I targetVertexId);
+
+ /**
+ * Set the edge value for the given target vertex id (if an edge to that
+ * vertex exists).
+ *
+ * @param targetVertexId Target vertex id
+ * @param edgeValue Edge value
+ */
+ void setEdgeValue(I targetVertexId, E edgeValue);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
deleted file mode 100644
index 5fdc2d2..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for {@link VertexEdges} implementations that provide efficient
- * random access to the edges given the target vertex id.
- * This version is for strict graphs (i.e. assumes no parallel edges).
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface StrictRandomAccessVertexEdges<I extends WritableComparable,
- E extends Writable> extends VertexEdges<I, E> {
- /**
- * Return the edge value for the given target vertex id (or null if there
- * is no edge pointing to it).
- *
- * @param targetVertexId Target vertex id
- * @return Edge value
- */
- E getEdgeValue(I targetVertexId);
-
- /**
- * Set the edge value for the given target vertex id (if an edge to that
- * vertex exists).
- *
- * @param targetVertexId Target vertex id
- * @param edgeValue Edge value
- */
- void setEdgeValue(I targetVertexId, E edgeValue);
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
deleted file mode 100644
index bb885b7..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for data structures that store out-edges for a vertex.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface VertexEdges<I extends WritableComparable, E extends Writable>
- extends Iterable<Edge<I, E>>, Writable {
- /**
- * Initialize the data structure and set the edges from an iterable.
- * This method (or one of the two alternatives) must be called
- * after instantiation, unless readFields() is called.
- * Note: whether parallel edges are allowed or not depends on the
- * implementation.
- *
- * @param edges Iterable of edges
- */
- void initialize(Iterable<Edge<I, E>> edges);
-
- /**
- * Initialize the data structure with the specified initial capacity.
- * This method (or one of the two alternatives) must be called
- * after instantiation, unless readFields() is called.
- *
- * @param capacity Initial capacity
- */
- void initialize(int capacity);
-
- /**
- * Initialize the data structure with the default initial capacity.
- * This method (or one of the two alternatives) must be called
- * after instantiation, unless readFields() is called.
- *
- */
- void initialize();
-
- /**
- * Add an edge.
- * Note: whether parallel edges are allowed or not depends on the
- * implementation.
- *
- * @param edge Edge to add
- */
- void add(Edge<I, E> edge);
-
- /**
- * Remove all edges to the given target vertex.
- * Note: the implementation will vary depending on whether parallel edges
- * are allowed or not.
- *
- * @param targetVertexId Target vertex id
- */
- void remove(I targetVertexId);
-
- /**
- * Return the number of edges.
- *
- * @return Number of edges
- */
- int size();
-}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
index ade25fe..61624e5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
@@ -21,13 +21,13 @@ package org.apache.giraph.graph;
import com.google.common.collect.UnmodifiableIterator;
import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.MultiRandomAccessVertexEdges;
+import org.apache.giraph.edge.MultiRandomAccessOutEdges;
import org.apache.giraph.edge.MutableEdge;
import org.apache.giraph.edge.MutableEdgesIterable;
import org.apache.giraph.edge.MutableEdgesWrapper;
-import org.apache.giraph.edge.MutableVertexEdges;
-import org.apache.giraph.edge.StrictRandomAccessVertexEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.MutableOutEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.edge.StrictRandomAccessOutEdges;
import org.apache.giraph.partition.PartitionContext;
import org.apache.giraph.worker.WorkerAggregatorUsage;
import org.apache.giraph.worker.WorkerContext;
@@ -57,7 +57,7 @@ public abstract class Vertex<I extends WritableComparable,
/** Vertex value. */
private V value;
/** Outgoing edges. */
- private VertexEdges<I, E> edges;
+ private OutEdges<I, E> edges;
/** If true, do not do anymore computation on this vertex. */
private boolean halt;
/** Global graph state **/
@@ -89,7 +89,7 @@ public abstract class Vertex<I extends WritableComparable,
public void initialize(I id, V value) {
this.id = id;
this.value = value;
- this.edges = getConf().createAndInitializeVertexEdges(0);
+ this.edges = getConf().createAndInitializeOutEdges(0);
}
/**
@@ -98,13 +98,13 @@ public abstract class Vertex<I extends WritableComparable,
* @param edges Iterable of edges
*/
public void setEdges(Iterable<Edge<I, E>> edges) {
- // If the iterable is actually an instance of VertexEdges,
+ // If the iterable is actually an instance of OutEdges,
// we simply take the reference.
- // Otherwise, we initialize a new VertexEdges.
- if (edges instanceof VertexEdges) {
- this.edges = (VertexEdges<I, E>) edges;
+ // Otherwise, we initialize a new OutEdges.
+ if (edges instanceof OutEdges) {
+ this.edges = (OutEdges<I, E>) edges;
} else {
- this.edges = getConf().createAndInitializeVertexEdges(edges);
+ this.edges = getConf().createAndInitializeOutEdges(edges);
}
}
@@ -195,14 +195,14 @@ public abstract class Vertex<I extends WritableComparable,
* @return An iterable of mutable out-edges
*/
public Iterable<MutableEdge<I, E>> getMutableEdges() {
- // If the VertexEdges implementation has a specialized mutable iterator,
+ // If the OutEdges implementation has a specialized mutable iterator,
// we use that; otherwise, we build a new data structure as we iterate
// over the current edges.
- if (edges instanceof MutableVertexEdges) {
+ if (edges instanceof MutableOutEdges) {
return new Iterable<MutableEdge<I, E>>() {
@Override
public Iterator<MutableEdge<I, E>> iterator() {
- return ((MutableVertexEdges<I, E>) edges).mutableIterator();
+ return ((MutableOutEdges<I, E>) edges).mutableIterator();
}
};
} else {
@@ -212,8 +212,9 @@ public abstract class Vertex<I extends WritableComparable,
/**
* If a {@link MutableEdgesWrapper} was used to provide a mutable iterator,
- * copy any remaining edges to the new {@link VertexEdges} data
- * structure and keep a direct reference to it (thus discarding the wrapper).
+ * copy any remaining edges to the new {@link org.apache.giraph.edge.OutEdges}
+ * data structure and keep a direct reference to it (thus discarding the
+ * wrapper).
* Called by the Giraph infrastructure after computation.
*/
public void unwrapMutableEdges() {
@@ -242,10 +243,10 @@ public abstract class Vertex<I extends WritableComparable,
* @return Edge value (or null if missing)
*/
public E getEdgeValue(I targetVertexId) {
- // If the VertexEdges implementation has a specialized random-access
+ // If the OutEdges implementation has a specialized random-access
// method, we use that; otherwise, we scan the edges.
- if (edges instanceof StrictRandomAccessVertexEdges) {
- return ((StrictRandomAccessVertexEdges<I, E>) edges)
+ if (edges instanceof StrictRandomAccessOutEdges) {
+ return ((StrictRandomAccessOutEdges<I, E>) edges)
.getEdgeValue(targetVertexId);
} else {
for (Edge<I, E> edge : edges) {
@@ -265,10 +266,10 @@ public abstract class Vertex<I extends WritableComparable,
* @param edgeValue Edge value
*/
public void setEdgeValue(I targetVertexId, E edgeValue) {
- // If the VertexEdges implementation has a specialized random-access
+ // If the OutEdges implementation has a specialized random-access
// method, we use that; otherwise, we scan the edges.
- if (edges instanceof StrictRandomAccessVertexEdges) {
- ((StrictRandomAccessVertexEdges<I, E>) edges).setEdgeValue(
+ if (edges instanceof StrictRandomAccessOutEdges) {
+ ((StrictRandomAccessOutEdges<I, E>) edges).setEdgeValue(
targetVertexId, edgeValue);
} else {
for (MutableEdge<I, E> edge : getMutableEdges()) {
@@ -291,10 +292,10 @@ public abstract class Vertex<I extends WritableComparable,
* @return Iterable of edge values
*/
public Iterable<E> getAllEdgeValues(final I targetVertexId) {
- // If the VertexEdges implementation has a specialized random-access
+ // If the OutEdges implementation has a specialized random-access
// method, we use that; otherwise, we scan the edges.
- if (edges instanceof MultiRandomAccessVertexEdges) {
- return ((MultiRandomAccessVertexEdges<I, E>) edges)
+ if (edges instanceof MultiRandomAccessOutEdges) {
+ return ((MultiRandomAccessOutEdges<I, E>) edges)
.getAllEdgeValues(targetVertexId);
} else {
return new Iterable<E>() {
@@ -405,7 +406,7 @@ public abstract class Vertex<I extends WritableComparable,
* @param value Vertex value
* @param edges Initial edges
*/
- public void addVertexRequest(I id, V value, VertexEdges<I, E> edges)
+ public void addVertexRequest(I id, V value, OutEdges<I, E> edges)
throws IOException {
Vertex<I, V, E, M> vertex = getConf().createVertex();
vertex.initialize(id, value, edges);
@@ -420,7 +421,7 @@ public abstract class Vertex<I extends WritableComparable,
* @param value Vertex value
*/
public void addVertexRequest(I id, V value) throws IOException {
- addVertexRequest(id, value, getConf().createAndInitializeVertexEdges());
+ addVertexRequest(id, value, getConf().createAndInitializeOutEdges());
}
/**
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
index 2cc4dba..48544b0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
@@ -72,7 +72,7 @@ public class PseudoRandomEdgeInputFormat
/** Current vertex id. */
private LongWritable currentVertexId = new LongWritable(-1);
/** Edges read for the current vertex. */
- private int currentVertexEdgesRead = 0;
+ private int currentOutEdgesRead = 0;
/** Target vertices of edges for current vertex. */
private Set<LongWritable> currentVertexDestVertices = Sets.newHashSet();
/** Random number generator for the current vertex (for consistency
@@ -131,20 +131,20 @@ public class PseudoRandomEdgeInputFormat
public boolean nextEdge() throws IOException, InterruptedException {
return totalSplitVertices > verticesRead + 1 ||
(totalSplitVertices == verticesRead + 1 &&
- edgesPerVertex > currentVertexEdgesRead);
+ edgesPerVertex > currentOutEdgesRead);
}
@Override
public LongWritable getCurrentSourceId() throws IOException,
InterruptedException {
- if (currentVertexEdgesRead == edgesPerVertex) {
+ if (currentOutEdgesRead == edgesPerVertex) {
++verticesRead;
currentVertexId = new LongWritable(-1);
}
if (currentVertexId.get() == -1) {
currentVertexId.set(startingVertexId + verticesRead);
- currentVertexEdgesRead = 0;
+ currentOutEdgesRead = 0;
// Seed on the vertex id to keep the vertex data the same when
// on different number of workers, but other parameters are the
// same.
@@ -162,7 +162,7 @@ public class PseudoRandomEdgeInputFormat
destVertexId.set(localEdgesHelper.generateDestVertex(
currentVertexId.get(), random));
} while (currentVertexDestVertices.contains(destVertexId));
- ++currentVertexEdgesRead;
+ ++currentOutEdgesRead;
currentVertexDestVertices.add(destVertexId);
if (LOG.isTraceEnabled()) {
LOG.trace("getCurrentEdge: Return edge (" + currentVertexId + ", " +
@@ -178,7 +178,7 @@ public class PseudoRandomEdgeInputFormat
@Override
public float getProgress() throws IOException, InterruptedException {
- return (verticesRead * edgesPerVertex + currentVertexEdgesRead) *
+ return (verticesRead * edgesPerVertex + currentOutEdgesRead) *
100.0f / (totalSplitVertices * edgesPerVertex);
}
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
index b27fcc8..c9390ba 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
@@ -20,8 +20,8 @@ package org.apache.giraph.io.formats;
import org.apache.giraph.bsp.BspInputSplit;
import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.edge.ReusableEdge;
-import org.apache.giraph.edge.VertexEdges;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.io.VertexInputFormat;
import org.apache.giraph.io.VertexReader;
@@ -121,8 +121,8 @@ public class PseudoRandomIntNullVertexInputFormat extends
Vertex<IntWritable, FloatWritable, NullWritable, ?> vertex =
getConf().createVertex();
int vertexId = startingVertexId + verticesRead;
- VertexEdges<IntWritable, NullWritable> edges =
- getConf().createVertexEdges();
+ OutEdges<IntWritable, NullWritable> edges =
+ getConf().createOutEdges();
edges.initialize(edgesPerVertex);
destVertices.clear();
for (int i = 0; i < edgesPerVertex; ++i) {