flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From g...@apache.org
Subject [1/2] flink git commit: [hotfix] [gelly] Update test graph generation
Date Thu, 13 Apr 2017 18:06:26 GMT
Repository: flink
Updated Branches:
  refs/heads/master e651df99f -> 43d3046bb


[hotfix] [gelly] Update test graph generation

In tests RMat graphs are now created by calling a function with the
desired scale and edge factor. Also updated the documentation for
generating test graphs using Gelly examples.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/603c1209
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/603c1209
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/603c1209

Branch: refs/heads/master
Commit: 603c120956989fdb004ff824ba3b62b6b62daa3c
Parents: e651df9
Author: Greg Hogan <code@greghogan.com>
Authored: Thu Apr 13 11:09:47 2017 -0400
Committer: Greg Hogan <code@greghogan.com>
Committed: Thu Apr 13 13:11:37 2017 -0400

----------------------------------------------------------------------
 .../org/apache/flink/graph/asm/AsmTestBase.java | 72 ++++++++++++++------
 .../annotate/directed/EdgeDegreesPairTest.java  |  2 +-
 .../directed/EdgeSourceDegreesTest.java         |  2 +-
 .../directed/EdgeTargetDegreesTest.java         |  2 +-
 .../annotate/directed/VertexDegreesTest.java    |  2 +-
 .../annotate/directed/VertexInDegreeTest.java   |  2 +-
 .../annotate/directed/VertexOutDegreeTest.java  |  2 +-
 .../annotate/undirected/EdgeDegreePairTest.java |  4 +-
 .../undirected/EdgeSourceDegreeTest.java        |  4 +-
 .../undirected/EdgeTargetDegreeTest.java        |  4 +-
 .../annotate/undirected/VertexDegreeTest.java   |  4 +-
 .../filter/undirected/MaximumDegreeTest.java    |  2 +-
 .../AverageClusteringCoefficientTest.java       |  2 +-
 .../GlobalClusteringCoefficientTest.java        |  2 +-
 .../LocalClusteringCoefficientTest.java         |  2 +-
 .../clustering/directed/TriadicCensusTest.java  |  4 +-
 .../directed/TriangleListingTest.java           |  2 +-
 .../AverageClusteringCoefficientTest.java       |  2 +-
 .../GlobalClusteringCoefficientTest.java        |  2 +-
 .../LocalClusteringCoefficientTest.java         |  2 +-
 .../undirected/TriadicCensusTest.java           |  2 +-
 .../undirected/TriangleListingTest.java         |  2 +-
 .../graph/library/link_analysis/HITSTest.java   |  2 +-
 .../library/link_analysis/PageRankTest.java     |  2 +-
 .../metric/directed/EdgeMetricsTest.java        |  2 +-
 .../metric/directed/VertexMetricsTest.java      |  2 +-
 .../metric/undirected/EdgeMetricsTest.java      |  2 +-
 .../metric/undirected/VertexMetricsTest.java    |  2 +-
 .../library/similarity/AdamicAdarTest.java      | 17 +----
 .../library/similarity/JaccardIndexTest.java    | 17 +----
 30 files changed, 84 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
index b057121..1811038 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java
@@ -54,11 +54,6 @@ public class AsmTestBase {
 
 	protected Graph<LongValue, NullValue, NullValue> emptyGraph;
 
-	// RMat graph
-	protected Graph<LongValue, NullValue, NullValue> directedRMatGraph;
-
-	protected Graph<LongValue, NullValue, NullValue> undirectedRMatGraph;
-
 	@Before
 	public void setup()
 			throws Exception {
@@ -66,7 +61,7 @@ public class AsmTestBase {
 		env.getConfig().enableObjectReuse();
 
 		// the "fish" graph
-		Object[][] edges = new Object[][] {
+		Object[][] edges = new Object[][]{
 			new Object[]{0, 1},
 			new Object[]{0, 2},
 			new Object[]{2, 1},
@@ -93,26 +88,59 @@ public class AsmTestBase {
 		// empty graph
 		emptyGraph = new EmptyGraph(env, emptyGraphVertexCount)
 			.generate();
+	}
 
-		// RMat graph
-		long rmatVertexCount = 1L << 10;
-		long rmatEdgeCount = 16 * rmatVertexCount;
-
-		Graph<LongValue, NullValue, NullValue> rmatGraph = new RMatGraph<>(env, new
JDKRandomGeneratorFactory(), rmatVertexCount, rmatEdgeCount)
-			.generate();
+	/**
+	 * Generate a directed RMat graph. Tests are usually run on a graph with
+	 * scale=10 and edgeFactor=16 but algorithms generating very large DataSets
+	 * require smaller input graphs.
+	 *
+	 * The examples program can write this graph as a CSV file for verifying
+	 * algorithm results with external libraries:
+	 *
+	 * ./bin/flink run examples/flink-gelly-examples_*.jar --algorithm EdgeList \
+	 *     --input RMatGraph --type long --simplify directed --scale $SCALE --edge_factor $EDGE_FACTOR
\
+	 *     --output csv --filename directedRMatGraph.csv
+	 *
+	 * @param scale vertices are generated in the range [0, 2<sup>scale</sup>)
+	 * @param edgeFactor the edge count is {@code edgeFactor} * 2<sup>scale</sup>
+	 * @return directed RMat graph
+	 * @throws Exception on error
+	 */
+	protected Graph<LongValue, NullValue, NullValue> directedRMatGraph(int scale, int
edgeFactor)
+			throws Exception {
+		long vertexCount = 1L << scale;
+		long edgeCount = edgeFactor * vertexCount;
 
-		/*
-			./bin/flink run -c org.apache.flink.graph.drivers.Graph500 flink-gelly-examples_2.10-1.2-SNAPSHOT.jar
\
-				--directed true --simplify true --scale 10 --edge_factor 16 --output csv --filename directedRMatGraph.csv
-		 */
-		directedRMatGraph = rmatGraph
+		return new RMatGraph<>(env, new JDKRandomGeneratorFactory(), vertexCount, edgeCount)
+			.generate()
 			.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue,
NullValue>());
+	}
+
+	/**
+	 * Generate an undirected RMat graph. Tests are usually run on a graph with
+	 * scale=10 and edgeFactor=16 but algorithms generating very large DataSets
+	 * require smaller input graphs.
+	 *
+	 * The examples program can write this graph as a CSV file for verifying
+	 * algorithm results with external libraries:
+	 *
+	 * ./bin/flink run examples/flink-gelly-examples_*.jar --algorithm EdgeList \
+	 *     --input RMatGraph --type long --simplify undirected --scale $SCALE --edge_factor
$EDGE_FACTOR \
+	 *     --output csv --filename undirectedRMatGraph.csv
+	 *
+	 * @param scale vertices are generated in the range [0, 2<sup>scale</sup>)
+	 * @param edgeFactor the edge count is {@code edgeFactor} * 2<sup>scale</sup>
+	 * @return undirected RMat graph
+	 * @throws Exception on error
+	 */
+	protected Graph<LongValue, NullValue, NullValue> undirectedRMatGraph(int scale, int
edgeFactor)
+			throws Exception {
+		long vertexCount = 1L << scale;
+		long edgeCount = edgeFactor * vertexCount;
 
-		/*
-			./bin/flink run -c org.apache.flink.graph.drivers.Graph500 flink-gelly-examples_2.10-1.2-SNAPSHOT.jar
\
-				--directed false --simplify true --scale 10 --edge_factor 16 --output csv --filename
undirectedRMatGraph.csv
-		 */
-		undirectedRMatGraph = rmatGraph
+		return new RMatGraph<>(env, new JDKRandomGeneratorFactory(), vertexCount, edgeCount)
+			.generate()
 			.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue,
NullValue>(false));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
index 0540f14..18b52aa 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeDegreesPairTest.java
@@ -57,7 +57,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple3<NullValue, Degrees, Degrees>>> degreesPair
= directedRMatGraph
+		DataSet<Edge<LongValue, Tuple3<NullValue, Degrees, Degrees>>> degreesPair
= directedRMatGraph(10, 16)
 			.run(new EdgeDegreesPair<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple3<NullValue, Degrees,
Degrees>>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
index 8d52889..097b9c8 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeSourceDegreesTest.java
@@ -57,7 +57,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> sourceDegrees =
directedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> sourceDegrees =
directedRMatGraph(10, 16)
 			.run(new EdgeSourceDegrees<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, Degrees>>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
index eb0b892..b082088 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/EdgeTargetDegreesTest.java
@@ -57,7 +57,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> targetDegrees =
directedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, Degrees>>> targetDegrees =
directedRMatGraph(10, 16)
 			.run(new EdgeTargetDegrees<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue, Degrees>>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
index 06737b5..d0aad8f 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreesTest.java
@@ -95,7 +95,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 	throws Exception {
-		DataSet<Vertex<LongValue, Degrees>> degrees = directedRMatGraph
+		DataSet<Vertex<LongValue, Degrees>> degrees = directedRMatGraph(10, 16)
 			.run(new VertexDegrees<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new ChecksumHashCode<Vertex<LongValue, Degrees>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java
index 95f83853..3cbcc74 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java
@@ -78,7 +78,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Vertex<LongValue, LongValue>> inDegree = directedRMatGraph
+		DataSet<Vertex<LongValue, LongValue>> inDegree = directedRMatGraph(10, 16)
 			.run(new VertexInDegree<LongValue, NullValue, NullValue>()
 				.setIncludeZeroDegreeVertices(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java
index 7da3d8d..b72f0ef 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java
@@ -78,7 +78,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Vertex<LongValue, LongValue>> outDegree = directedRMatGraph
+		DataSet<Vertex<LongValue, LongValue>> outDegree = directedRMatGraph(10, 16)
 			.run(new VertexOutDegree<LongValue, NullValue, NullValue>()
 				.setIncludeZeroDegreeVertices(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
index 476b3fe..c65ef2d 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java
@@ -69,7 +69,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnSourceId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnSourceId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeDegreePair<LongValue, NullValue, NullValue>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Edge<LongValue, Tuple3<NullValue,
LongValue, LongValue>>>()
@@ -79,7 +79,7 @@ extends AsmTestBase {
 		assertEquals(20884, checksumOnSourceId.getCount());
 		assertEquals(0x00000001e051efe4L, checksumOnSourceId.getChecksum());
 
-		DataSet<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnTargetId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple3<NullValue, LongValue, LongValue>>> degreePairOnTargetId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeDegreePair<LongValue, NullValue, NullValue>()
 				.setReduceOnTargetId(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
index 0dc2178..2fd0b19 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java
@@ -69,7 +69,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnSourceId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnSourceId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeSourceDegree<LongValue, NullValue, NullValue>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue,
LongValue>>>()
@@ -79,7 +79,7 @@ extends AsmTestBase {
 		assertEquals(20884, checksumOnSourceId.getCount());
 		assertEquals(0x000000019d8f0070L, checksumOnSourceId.getChecksum());
 
-		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnTargetId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> sourceDegreeOnTargetId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeSourceDegree<LongValue, NullValue, NullValue>()
 				.setReduceOnTargetId(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
index b14ddc0..34aca35 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java
@@ -69,7 +69,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> targetDegreeOnTargetId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> targetDegreeOnTargetId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeSourceDegree<LongValue, NullValue, NullValue>());
 
 		Checksum checksumOnTargetId = new ChecksumHashCode<Edge<LongValue, Tuple2<NullValue,
LongValue>>>()
@@ -79,7 +79,7 @@ extends AsmTestBase {
 		assertEquals(20884, checksumOnTargetId.getCount());
 		assertEquals(0x000000019d8f0070L, checksumOnTargetId.getChecksum());
 
-		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> targetDegreeOnSourceId
= undirectedRMatGraph
+		DataSet<Edge<LongValue, Tuple2<NullValue, LongValue>>> targetDegreeOnSourceId
= undirectedRMatGraph(10, 16)
 			.run(new EdgeTargetDegree<LongValue, NullValue, NullValue>()
 				.setReduceOnSourceId(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
index 102beae..c157cc1 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java
@@ -104,7 +104,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Vertex<LongValue, LongValue>> degreeOnSourceId = undirectedRMatGraph
+		DataSet<Vertex<LongValue, LongValue>> degreeOnSourceId = undirectedRMatGraph(10,
16)
 			.run(new VertexDegree<LongValue, NullValue, NullValue>());
 
 		Checksum checksumOnSourceId = new ChecksumHashCode<Vertex<LongValue, LongValue>>()
@@ -114,7 +114,7 @@ extends AsmTestBase {
 		assertEquals(902, checksumOnSourceId.getCount());
 		assertEquals(0x0000000000e1fb30L, checksumOnSourceId.getChecksum());
 
-		DataSet<Vertex<LongValue, LongValue>> degreeOnTargetId = undirectedRMatGraph
+		DataSet<Vertex<LongValue, LongValue>> degreeOnTargetId = undirectedRMatGraph(10,
16)
 			.run(new VertexDegree<LongValue, NullValue, NullValue>()
 				.setReduceOnTargetId(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
index 55f7743..f017750 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/filter/undirected/MaximumDegreeTest.java
@@ -62,7 +62,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		Checksum checksum = undirectedRMatGraph
+		Checksum checksum = undirectedRMatGraph(10, 16)
 			.run(new MaximumDegree<LongValue, NullValue, NullValue>(16))
 			.run(new ChecksumHashCode<LongValue, NullValue, NullValue>())
 			.execute();

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/AverageClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/AverageClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/AverageClusteringCoefficientTest.java
index 9de9bac..db0a8a1 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/AverageClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/AverageClusteringCoefficientTest.java
@@ -73,7 +73,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(902, 297.152607);
 
 		Result averageClusteringCoefficient = new AverageClusteringCoefficient<LongValue, NullValue,
NullValue>()
-			.run(directedRMatGraph)
+			.run(directedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult.getNumberOfVertices(), averageClusteringCoefficient.getNumberOfVertices());

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/GlobalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/GlobalClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/GlobalClusteringCoefficientTest.java
index 70ca6d7..50006fc 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/GlobalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/GlobalClusteringCoefficientTest.java
@@ -76,7 +76,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(1003442, 225147);
 
 		Result globalClusteringCoefficient = new GlobalClusteringCoefficient<LongValue, NullValue,
NullValue>()
-			.run(directedRMatGraph)
+			.run(directedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, globalClusteringCoefficient);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
index 9aaa062..f1dd57b 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/LocalClusteringCoefficientTest.java
@@ -75,7 +75,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		DataSet<Result<LongValue>> cc = directedRMatGraph
+		DataSet<Result<LongValue>> cc = directedRMatGraph(10, 16)
 			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new org.apache.flink.graph.asm.dataset.ChecksumHashCode<Result<LongValue>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriadicCensusTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriadicCensusTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriadicCensusTest.java
index 965f602..a4d1560 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriadicCensusTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriadicCensusTest.java
@@ -88,7 +88,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(113_435_893, 0, 7_616_063, 0, 0, 0, 0, 0, 0, 0, 778_295,
0, 0, 0, 0, 75_049);
 
 		Result triadCensus = new TriadicCensus<LongValue, NullValue, NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, triadCensus);
@@ -115,7 +115,7 @@ extends AsmTestBase {
 			7_587, 15_178, 17_368, 4_951);
 
 		Result triadCensus = new TriadicCensus<LongValue, NullValue, NullValue>()
-			.run(directedRMatGraph)
+			.run(directedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, triadCensus);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
index 6ae9b90..3999959 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/directed/TriangleListingTest.java
@@ -72,7 +72,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		DataSet<Result<LongValue>> tl = directedRMatGraph
+		DataSet<Result<LongValue>> tl = directedRMatGraph(10, 16)
 			.run(new TriangleListing<LongValue, NullValue, NullValue>()
 				.setSortTriangleVertices(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/AverageClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/AverageClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/AverageClusteringCoefficientTest.java
index 34fda17..ee34ac6 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/AverageClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/AverageClusteringCoefficientTest.java
@@ -73,7 +73,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(902, 380.40109);
 
 		Result averageClusteringCoefficient = new AverageClusteringCoefficient<LongValue, NullValue,
NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult.getNumberOfVertices(), averageClusteringCoefficient.getNumberOfVertices());

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/GlobalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/GlobalClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/GlobalClusteringCoefficientTest.java
index 71ec2a6..1a73ce1 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/GlobalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/GlobalClusteringCoefficientTest.java
@@ -76,7 +76,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(1003442, 225147);
 
 		Result globalClusteringCoefficient = new GlobalClusteringCoefficient<LongValue, NullValue,
NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, globalClusteringCoefficient);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
index e9097dc..2775a00 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java
@@ -76,7 +76,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		DataSet<Result<LongValue>> cc = undirectedRMatGraph
+		DataSet<Result<LongValue>> cc = undirectedRMatGraph(10, 16)
 			.run(new LocalClusteringCoefficient<LongValue, NullValue, NullValue>());
 
 		Checksum checksum = new ChecksumHashCode<Result<LongValue>>()

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriadicCensusTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriadicCensusTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriadicCensusTest.java
index 6d34d95..87b4824 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriadicCensusTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriadicCensusTest.java
@@ -86,7 +86,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(113_435_893, 7_616_063, 778_295, 75_049);
 
 		Result triadCensus = new TriadicCensus<LongValue, NullValue, NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, triadCensus);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
index dee066c..afbd740 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java
@@ -68,7 +68,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		DataSet<Result<LongValue>> tl = undirectedRMatGraph
+		DataSet<Result<LongValue>> tl = undirectedRMatGraph(10, 16)
 			.run(new TriangleListing<LongValue, NullValue, NullValue>()
 				.setSortTriangleVertices(true));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/HITSTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/HITSTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/HITSTest.java
index e9db838..03334da 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/HITSTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/HITSTest.java
@@ -110,7 +110,7 @@ extends AsmTestBase {
 	@Test
 	public void testWithRMatGraph()
 			throws Exception {
-		DataSet<Result<LongValue>> hits = directedRMatGraph
+		DataSet<Result<LongValue>> hits = directedRMatGraph(10, 16)
 			.run(new HITS<LongValue, NullValue, NullValue>(0.000001));
 
 		Map<Long, Result<LongValue>> results = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/PageRankTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/PageRankTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/PageRankTest.java
index 082b6ad..fc7e485 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/PageRankTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/link_analysis/PageRankTest.java
@@ -104,7 +104,7 @@ extends AsmTestBase {
 	public void testWithRMatGraph()
 			throws Exception {
 		DataSet<Result<LongValue>> pr = new PageRank<LongValue, NullValue, NullValue>(DAMPING_FACTOR,
0.000001)
-			.run(directedRMatGraph);
+			.run(directedRMatGraph(10, 16));
 
 		Map<Long, Result<LongValue>> results = new HashMap<>();
 		for (Result<LongValue> result :  new Collect<Result<LongValue>>().run(pr).execute())
{

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java
index 9711cc0..117b3ae 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java
@@ -80,7 +80,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(107817, 315537, 820, 3822);
 
 		Result withoutZeroDegreeVertices = new EdgeMetrics<LongValue, NullValue, NullValue>()
-			.run(directedRMatGraph)
+			.run(directedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, withoutZeroDegreeVertices);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/VertexMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/VertexMetricsTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/VertexMetricsTest.java
index eb25816..54301f5 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/VertexMetricsTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/VertexMetricsTest.java
@@ -97,7 +97,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(902, 8875, 1567, 1003442, 463, 334, 342, 106953);
 
 		Result withoutZeroDegreeVertices = new VertexMetrics<LongValue, NullValue, NullValue>()
-			.run(directedRMatGraph)
+			.run(directedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, withoutZeroDegreeVertices);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java
index cb75331..b4e9f95 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java
@@ -80,7 +80,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(107817, 315537, 820, 3822);
 
 		Result withoutZeroDegreeVertices = new EdgeMetrics<LongValue, NullValue, NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, withoutZeroDegreeVertices);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/VertexMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/VertexMetricsTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/VertexMetricsTest.java
index 6eecc42..848ad79 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/VertexMetricsTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/VertexMetricsTest.java
@@ -97,7 +97,7 @@ extends AsmTestBase {
 		Result expectedResult = new Result(902, 10442, 1003442, 463, 106953);
 
 		Result withoutZeroDegreeVertices = new VertexMetrics<LongValue, NullValue, NullValue>()
-			.run(undirectedRMatGraph)
+			.run(undirectedRMatGraph(10, 16))
 			.execute();
 
 		assertEquals(expectedResult, withoutZeroDegreeVertices);

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
index 55d9abb..76b28da 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/AdamicAdarTest.java
@@ -18,14 +18,8 @@
 
 package org.apache.flink.graph.library.similarity;
 
-import org.apache.commons.math3.random.JDKRandomGenerator;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.asm.AsmTestBase;
-import org.apache.flink.graph.asm.simple.undirected.Simplify;
-import org.apache.flink.graph.generator.RMatGraph;
-import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory;
-import org.apache.flink.graph.generator.random.RandomGenerableFactory;
 import org.apache.flink.graph.library.similarity.AdamicAdar.Result;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.IntValue;
@@ -117,16 +111,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		long vertexCount = 1 << 8;
-		long edgeCount = 8 * vertexCount;
-
-		RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-		Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount,
edgeCount)
-			.generate()
-			.run(new Simplify<LongValue, NullValue, NullValue>(false));
-
-		DataSet<Result<LongValue>> aa = graph
+		DataSet<Result<LongValue>> aa = undirectedRMatGraph(8, 8)
 			.run(new AdamicAdar<LongValue, NullValue, NullValue>());
 
 		assertEquals(13954, aa.count());

http://git-wip-us.apache.org/repos/asf/flink/blob/603c1209/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
index d69a441..128ee70 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/similarity/JaccardIndexTest.java
@@ -18,16 +18,10 @@
 
 package org.apache.flink.graph.library.similarity;
 
-import org.apache.commons.math3.random.JDKRandomGenerator;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.asm.AsmTestBase;
 import org.apache.flink.graph.asm.dataset.ChecksumHashCode;
 import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum;
-import org.apache.flink.graph.asm.simple.undirected.Simplify;
-import org.apache.flink.graph.generator.RMatGraph;
-import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory;
-import org.apache.flink.graph.generator.random.RandomGenerableFactory;
 import org.apache.flink.graph.library.similarity.JaccardIndex.Result;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.IntValue;
@@ -116,16 +110,7 @@ extends AsmTestBase {
 	@Test
 	public void testRMatGraph()
 			throws Exception {
-		long vertexCount = 1 << 8;
-		long edgeCount = 8 * vertexCount;
-
-		RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-		Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount,
edgeCount)
-			.generate()
-			.run(new Simplify<LongValue, NullValue, NullValue>(false));
-
-		DataSet<Result<LongValue>> ji = graph
+		DataSet<Result<LongValue>> ji = undirectedRMatGraph(8, 8)
 			.run(new JaccardIndex<LongValue, NullValue, NullValue>()
 				.setGroupSize(4));
 


Mime
View raw message