You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gr...@apache.org on 2017/03/31 16:35:24 UTC

[1/6] flink git commit: [FLINK-2814] [optimizer] DualInputPlanNode cannot be cast to SingleInputPlanNode

Repository: flink
Updated Branches:
  refs/heads/master 2313a74e2 -> a48357db8


[FLINK-2814] [optimizer] DualInputPlanNode cannot be cast to SingleInputPlanNode

WorksetIterationNode#instantiate loops over all solution and work set
candidates. Since the solution set reference is modified in place when
the predecessor node can be used in its place, swith this variable to
the inner loop.

This closes #3563


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

Branch: refs/heads/master
Commit: 43158a86d9c3f3d6e3d612c8871625e9d22e7ba3
Parents: 2313a74
Author: Greg Hogan <co...@greghogan.com>
Authored: Fri Mar 17 12:09:34 2017 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 31 11:05:58 2017 -0400

----------------------------------------------------------------------
 .../org/apache/flink/optimizer/dag/WorksetIterationNode.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/43158a86/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
index d7ccaca..e62e727 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
@@ -431,8 +431,8 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 		LocalProperties lp = LocalProperties.EMPTY.addUniqueFields(this.solutionSetKeyFields);
 		
 		// take all combinations of solution set delta and workset plans
-		for (PlanNode solutionSetCandidate : solutionSetDeltaCandidates) {
-			for (PlanNode worksetCandidate : worksetCandidates) {
+		for (PlanNode worksetCandidate : worksetCandidates) {
+			for (PlanNode solutionSetCandidate : solutionSetDeltaCandidates) {
 				// check whether they have the same operator at their latest branching point
 				if (this.singleRoot.areBranchCompatible(solutionSetCandidate, worksetCandidate)) {
 					


[4/6] flink git commit: [FLINK-5912] [gelly] Inputs for CSV and graph generators

Posted by gr...@apache.org.
[FLINK-5912] [gelly] Inputs for CSV and graph generators

Create Input classes for reading graphs from CSV as well as for each of
the graph generators.

This closes #3626


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

Branch: refs/heads/master
Commit: ded25be4d8fc51f2a58cbd3264daffe48dca6b04
Parents: 963f46e
Author: Greg Hogan <co...@greghogan.com>
Authored: Mon Mar 27 10:08:59 2017 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 31 11:16:05 2017 -0400

----------------------------------------------------------------------
 .../apache/flink/graph/drivers/input/CSV.java   | 115 ++++++++++++
 .../graph/drivers/input/CompleteGraph.java      |  60 +++++++
 .../flink/graph/drivers/input/CycleGraph.java   |  60 +++++++
 .../flink/graph/drivers/input/EmptyGraph.java   |  55 ++++++
 .../flink/graph/drivers/input/GridGraph.java    | 144 +++++++++++++++
 .../graph/drivers/input/HypercubeGraph.java     |  60 +++++++
 .../flink/graph/drivers/input/PathGraph.java    |  60 +++++++
 .../flink/graph/drivers/input/RMatGraph.java    | 174 +++++++++++++++++++
 .../graph/drivers/input/SingletonEdgeGraph.java |  60 +++++++
 .../flink/graph/drivers/input/StarGraph.java    |  60 +++++++
 .../graph/drivers/parameter/Parameterized.java  |   2 +-
 .../drivers/parameter/ParameterizedBase.java    |   2 +-
 .../flink/graph/drivers/parameter/Simplify.java | 137 +++++++++++++++
 .../graph/drivers/parameter/SimplifyTest.java   |  56 ++++++
 14 files changed, 1043 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CSV.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CSV.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CSV.java
new file mode 100644
index 0000000..58b65b6
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CSV.java
@@ -0,0 +1,115 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.commons.lang3.text.WordUtils;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.CsvInputFormat;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.GraphCsvReader;
+import org.apache.flink.graph.drivers.parameter.ChoiceParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.graph.drivers.parameter.Simplify;
+import org.apache.flink.graph.drivers.parameter.StringParameter;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.types.StringValue;
+
+/**
+ * Read a {@link Graph} from a CSV file using {@link IntValue},
+ * {@link LongValue}, or {@link StringValue} keys.
+ *
+ * @param <K> key type
+ */
+public class CSV<K extends Comparable<K>>
+extends ParameterizedBase
+implements Input<K, NullValue, NullValue> {
+
+	private static final String INTEGER = "integer";
+
+	private static final String LONG = "long";
+
+	private static final String STRING = "string";
+
+	private ChoiceParameter type = new ChoiceParameter(this, "type")
+		.setDefaultValue(INTEGER)
+		.addChoices(LONG, STRING);
+
+	private StringParameter inputFilename = new StringParameter(this, "input_filename");
+
+	private StringParameter commentPrefix = new StringParameter(this, "comment_prefix")
+		.setDefaultValue("#");
+
+	private StringParameter lineDelimiter = new StringParameter(this, "input_line_delimiter")
+		.setDefaultValue(CsvInputFormat.DEFAULT_LINE_DELIMITER);
+
+	private StringParameter fieldDelimiter = new StringParameter(this, "input_field_delimiter")
+		.setDefaultValue(CsvInputFormat.DEFAULT_FIELD_DELIMITER);
+
+	private Simplify simplify = new Simplify(this);
+
+	@Override
+	public String getName() {
+		return CSV.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return WordUtils.capitalize(getName()) + WordUtils.capitalize(type.getValue()) + " (" + inputFilename + ")";
+	}
+
+	/**
+	 * Generate the graph as configured.
+	 *
+	 * @param env execution environment
+	 * @return input graph
+	 */
+	public Graph<K, NullValue, NullValue> create(ExecutionEnvironment env) throws Exception {
+		GraphCsvReader reader = Graph.fromCsvReader(inputFilename.getValue(), env)
+			.ignoreCommentsEdges(commentPrefix.getValue())
+			.lineDelimiterEdges(lineDelimiter.getValue())
+			.fieldDelimiterEdges(fieldDelimiter.getValue());
+
+		Graph<K, NullValue, NullValue> graph;
+
+		switch (type.getValue()) {
+			case INTEGER:
+				graph = (Graph<K, NullValue, NullValue>) reader
+					.keyType(IntValue.class);
+				break;
+
+			case LONG:
+				graph = (Graph<K, NullValue, NullValue>) reader
+					.keyType(LongValue.class);
+				break;
+
+			case STRING:
+				graph = (Graph<K, NullValue, NullValue>) reader
+					.keyType(StringValue.class);
+				break;
+
+			default:
+				throw new ProgramParametrizationException("Unknown type '" + type.getValue() + "'");
+		}
+
+		return simplify.simplify(graph);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CompleteGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CompleteGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CompleteGraph.java
new file mode 100644
index 0000000..c31c5a8
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CompleteGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.CompleteGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.CompleteGraph}.
+ */
+public class CompleteGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexCount = new LongParameter(this, "vertex_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return CompleteGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexCount.getValue() + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.CompleteGraph(env, vertexCount.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CycleGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CycleGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CycleGraph.java
new file mode 100644
index 0000000..df66dab
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/CycleGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.CycleGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.CycleGraph}.
+ */
+public class CycleGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexCount = new LongParameter(this, "vertex_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return CycleGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexCount + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.CycleGraph(env, vertexCount.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/EmptyGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/EmptyGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/EmptyGraph.java
new file mode 100644
index 0000000..794317b
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/EmptyGraph.java
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.graph.generator.EmptyGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate an {@link org.apache.flink.graph.generator.EmptyGraph}.
+ */
+public class EmptyGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexCount = new LongParameter(this, "vertex_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	@Override
+	public String getName() {
+		return EmptyGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexCount + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.EmptyGraph(env, vertexCount.getValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/GridGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/GridGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/GridGraph.java
new file mode 100644
index 0000000..d502215
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/GridGraph.java
@@ -0,0 +1,144 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.GridGraph}.
+ */
+public class GridGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private static final String PREFIX = "dim";
+
+	private List<Dimension> dimensions = new ArrayList<>();
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return GridGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getUsage() {
+		return "--dim0 size:wrap_endpoints [--dim1 size:wrap_endpoints [--dim2 ...]]" + super.getUsage();
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) throws ProgramParametrizationException {
+		super.configure(parameterTool);
+
+		// add dimensions as ordered by dimension ID (dim0, dim1, dim2, ...)
+
+		Map<Integer, String> dimensionMap = new TreeMap<>();
+
+		// first parse all dimensions into a sorted map
+		for (Map.Entry<String, String> entry : parameterTool.toMap().entrySet()) {
+			if (entry.getKey().startsWith(PREFIX)) {
+				int dimensionId = Integer.parseInt(entry.getKey().substring(PREFIX.length()));
+				dimensionMap.put(dimensionId, entry.getValue());
+			}
+		}
+
+		// then store dimensions in order
+		for (String field : dimensionMap.values()) {
+			dimensions.add(new Dimension(field));
+		}
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + dimensions + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		org.apache.flink.graph.generator.GridGraph graph = new org.apache.flink.graph.generator.GridGraph(env);
+
+		for (Dimension dimension : dimensions) {
+			graph.addDimension(dimension.size, dimension.wrapEndpoints);
+		}
+
+		return graph
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+
+	/**
+	 * Stores and parses the size and endpoint wrapping configuration for a
+	 * {@link org.apache.flink.graph.generator.GridGraph} dimension.
+	 */
+	private static class Dimension {
+		private long size;
+
+		private boolean wrapEndpoints;
+
+		/**
+		 * Configuration string to be parsed. The size integer and endpoint
+		 * wrapping boolean must be separated by a colon.
+		 *
+		 * @param field configuration string
+		 */
+		public Dimension(String field) {
+			ProgramParametrizationException exception = new ProgramParametrizationException("Grid dimension must use " +
+				"a colon to separate the integer size and boolean indicating whether the dimension endpoints are " +
+				"connected: '" + field + "'");
+
+			if (! field.contains(":")) {
+				throw exception;
+			}
+
+			String[] parts = field.split(":");
+
+			if (parts.length != 2) {
+				throw exception;
+			}
+
+			try {
+				size = Long.parseLong(parts[0]);
+				wrapEndpoints = Boolean.parseBoolean(parts[1]);
+			} catch(NumberFormatException ex) {
+				throw exception;
+			}
+		}
+
+		@Override
+		public String toString() {
+			return Long.toString(size) + (wrapEndpoints ? "+" : "\u229e");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/HypercubeGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/HypercubeGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/HypercubeGraph.java
new file mode 100644
index 0000000..3b87b00
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/HypercubeGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.HypercubeGraph.MINIMUM_DIMENSIONS;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.HypercubeGraph}.
+ */
+public class HypercubeGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter dimensions = new LongParameter(this, "dimensions")
+		.setMinimumValue(MINIMUM_DIMENSIONS);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return HypercubeGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + dimensions + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.HypercubeGraph(env, dimensions.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/PathGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/PathGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/PathGraph.java
new file mode 100644
index 0000000..968628c
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/PathGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.PathGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.PathGraph}.
+ */
+public class PathGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexCount = new LongParameter(this, "vertex_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return PathGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexCount + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.PathGraph(env, vertexCount.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/RMatGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/RMatGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/RMatGraph.java
new file mode 100644
index 0000000..e4e6a4c
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/RMatGraph.java
@@ -0,0 +1,174 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.commons.lang3.text.WordUtils;
+import org.apache.commons.math3.random.JDKRandomGenerator;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.asm.translate.TranslateGraphIds;
+import org.apache.flink.graph.asm.translate.translators.LongValueToStringValue;
+import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
+import org.apache.flink.graph.drivers.parameter.BooleanParameter;
+import org.apache.flink.graph.drivers.parameter.ChoiceParameter;
+import org.apache.flink.graph.drivers.parameter.DoubleParameter;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.graph.drivers.parameter.Simplify;
+import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory;
+import org.apache.flink.graph.generator.random.RandomGenerableFactory;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.types.StringValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+
+/**
+ * Generate an {@code RMatGraph} with {@link IntValue}, {@link LongValue},
+ * or {@link StringValue} keys.
+ *
+ * @see org.apache.flink.graph.generator.RMatGraph
+ *
+ * @param <K> key type
+ */
+public class RMatGraph<K extends Comparable<K>>
+extends ParameterizedBase
+implements Input<K, NullValue, NullValue> {
+
+	private static final String INTEGER = "integer";
+
+	private static final String LONG = "long";
+
+	private static final String STRING = "string";
+
+	private ChoiceParameter type = new ChoiceParameter(this, "type")
+		.setDefaultValue(INTEGER)
+		.addChoices(LONG, STRING);
+
+	// generate graph with 2^scale vertices
+	private LongParameter scale = new LongParameter(this, "scale")
+		.setDefaultValue(10)
+		.setMinimumValue(1);
+
+	// generate graph with edgeFactor * 2^scale edges
+	private LongParameter edgeFactor = new LongParameter(this, "edge_factor")
+		.setDefaultValue(16)
+		.setMinimumValue(1);
+
+	// matrix parameters "a", "b", "c", and implicitly "d = 1 - a - b - c"
+	// describe the skew in the recursive matrix
+	private DoubleParameter a = new DoubleParameter(this, "a")
+		.setDefaultValue(org.apache.flink.graph.generator.RMatGraph.DEFAULT_A)
+		.setMinimumValue(0.0, false);
+
+	private DoubleParameter b = new DoubleParameter(this, "b")
+		.setDefaultValue(org.apache.flink.graph.generator.RMatGraph.DEFAULT_B)
+		.setMinimumValue(0.0, false);
+
+	private DoubleParameter c = new DoubleParameter(this, "c")
+		.setDefaultValue(org.apache.flink.graph.generator.RMatGraph.DEFAULT_C)
+		.setMinimumValue(0.0, false);
+
+	// noise randomly pertubates the matrix parameters for each successive bit
+	// for each generated edge
+	private BooleanParameter noiseEnabled = new BooleanParameter(this, "noise_enabled");
+
+	private DoubleParameter noise = new DoubleParameter(this, "noise")
+		.setDefaultValue(org.apache.flink.graph.generator.RMatGraph.DEFAULT_NOISE)
+		.setMinimumValue(0.0, true)
+		.setMaximumValue(2.0, true);
+
+	private Simplify simplify = new Simplify(this);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return RMatGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + WordUtils.capitalize(type.getValue()) +
+			" (s" + scale + "e" + edgeFactor + simplify.getShortString() + ")";
+	}
+
+	/**
+	 * Generate the graph as configured.
+	 *
+	 * @param env Flink execution environment
+	 * @return input graph
+	 */
+	public Graph<K, NullValue, NullValue> create(ExecutionEnvironment env) throws Exception {
+		int lp = littleParallelism.getValue().intValue();
+
+		RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
+
+		long vertexCount = 1L << scale.getValue();
+		long edgeCount = vertexCount * edgeFactor.getValue();
+
+		Graph<LongValue, NullValue, NullValue> rmatGraph = new org.apache.flink.graph.generator.RMatGraph<>(
+				env, rnd, vertexCount, edgeCount)
+			.setConstants(a.getValue().floatValue(), b.getValue().floatValue(), c.getValue().floatValue())
+			.setNoise(noiseEnabled.getValue(), noise.getValue().floatValue())
+			.setParallelism(lp)
+			.generate();
+
+		Graph<K, NullValue, NullValue> graph;
+
+		switch (type.getValue()) {
+			case INTEGER:
+				if (scale.getValue() > 32) {
+					throw new ProgramParametrizationException(
+						"Scale '" + scale.getValue() + "' must be no greater than 32 for type 'integer'");
+				}
+				graph = (Graph<K, NullValue, NullValue>) rmatGraph
+					.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue()));
+				break;
+
+			case LONG:
+				if (scale.getValue() > 64) {
+					throw new ProgramParametrizationException(
+						"Scale '" + scale.getValue() + "' must be no greater than 64 for type 'long'");
+				}
+				graph = (Graph<K, NullValue, NullValue>) rmatGraph;
+				break;
+
+			case STRING:
+				// scale bound is same as LONG since keys are generated as LongValue
+				if (scale.getValue() > 64) {
+					throw new ProgramParametrizationException(
+						"Scale '" + scale.getValue() + "' must be no greater than 64 for type 'string'");
+				}
+				graph = (Graph<K, NullValue, NullValue>) rmatGraph
+					.run(new TranslateGraphIds<LongValue, StringValue, NullValue, NullValue>(new LongValueToStringValue()));
+				break;
+
+			default:
+				throw new ProgramParametrizationException("Unknown type '" + type.getValue() + "'");
+		}
+
+		// simplify *after* the translation from LongValue to IntValue or
+		// StringValue to improve the performance of the simplify operators
+		return simplify.simplify(graph);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/SingletonEdgeGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/SingletonEdgeGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/SingletonEdgeGraph.java
new file mode 100644
index 0000000..502826f
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/SingletonEdgeGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.PathGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.SingletonEdgeGraph}.
+ */
+public class SingletonEdgeGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexPairCount = new LongParameter(this, "vertex_pair_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return SingletonEdgeGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexPairCount + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.SingletonEdgeGraph(env, vertexPairCount.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/StarGraph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/StarGraph.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/StarGraph.java
new file mode 100644
index 0000000..b794f5c
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/input/StarGraph.java
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.graph.drivers.input;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+import static org.apache.flink.graph.generator.StarGraph.MINIMUM_VERTEX_COUNT;
+
+/**
+ * Generate a {@link org.apache.flink.graph.generator.StarGraph}.
+ */
+public class StarGraph
+extends ParameterizedBase
+implements Input<LongValue, NullValue, NullValue> {
+
+	private LongParameter vertexCount = new LongParameter(this, "vertex_count")
+		.setMinimumValue(MINIMUM_VERTEX_COUNT);
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return StarGraph.class.getSimpleName();
+	}
+
+	@Override
+	public String getIdentity() {
+		return getName() + " (" + vertexCount + ")";
+	}
+
+	@Override
+	public Graph<LongValue, NullValue, NullValue> create(ExecutionEnvironment env) {
+		return new org.apache.flink.graph.generator.StarGraph(env, vertexCount.getValue())
+			.setParallelism(littleParallelism.getValue().intValue())
+			.generate();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameterized.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameterized.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameterized.java
index b24f8cf..7b291be 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameterized.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Parameterized.java
@@ -39,7 +39,7 @@ public interface Parameterized {
 	 *
 	 * @return command-line documentation string
 	 */
-	String getParameterization();
+	String getUsage();
 
 	/**
 	 * Read parameter values from the command-line arguments.

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
index 3b9b80a..5f36ff5 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/ParameterizedBase.java
@@ -45,7 +45,7 @@ implements Parameterized {
 	}
 
 	@Override
-	public String getParameterization() {
+	public String getUsage() {
 		StrBuilder strBuilder = new StrBuilder();
 
 		// print parameters as ordered list

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
new file mode 100644
index 0000000..3e9fd9a
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/Simplify.java
@@ -0,0 +1,137 @@
+/*
+ * 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.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.client.program.ProgramParametrizationException;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.parameter.Simplify.Ordering;
+import org.apache.flink.types.NullValue;
+
+/**
+ * A simple graph has no self-loops (edges where the source and target vertices
+ * are the same) and no duplicate edges. Flink stores an undirected graph as
+ * a directed graph where each undirected edge is represented by a directed
+ * edge in each direction.
+ *
+ * This {@link Parameter} indicates whether to simplify the graph and if the
+ * graph should be directed or undirected.
+ */
+public class Simplify
+implements Parameter<Ordering> {
+
+	public enum Ordering {
+		// leave the graph unchanged
+		NONE,
+
+		// create a simple, directed graph
+		DIRECTED,
+
+		// create a simple, undirected graph
+		UNDIRECTED,
+
+		// create a simple, undirected graph
+		// remove input edges where source < target before symmetrizing the graph
+		UNDIRECTED_CLIP_AND_FLIP,
+	}
+
+	private Ordering value;
+
+	/**
+	 * Add this parameter to the list of parameters stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 */
+	public Simplify(ParameterizedBase owner) {
+		owner.addParameter(this);
+	}
+
+	@Override
+	public String getUsage() {
+		return "[--simplify <directed | undirected [--clip_and_flip]>]";
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		String ordering = parameterTool.get("simplify");
+
+		if (ordering == null) {
+			value = Ordering.NONE;
+		} else {
+			switch (ordering.toLowerCase()) {
+				case "directed":
+					value = Ordering.DIRECTED;
+					break;
+				case "undirected":
+					value = parameterTool.has("clip_and_flip") ? Ordering.UNDIRECTED_CLIP_AND_FLIP : Ordering.UNDIRECTED;
+					break;
+				default:
+					throw new ProgramParametrizationException(
+						"Expected 'directed' or 'undirected' ordering but received '" + ordering + "'");
+			}
+		}
+	}
+
+	@Override
+	public Ordering getValue() {
+		return value;
+	}
+
+	/**
+	 * Simplify the given graph based on the configured value.
+	 *
+	 * @param graph input graph
+	 * @param <T> graph key type
+	 * @return output graph
+	 * @throws Exception on error
+	 */
+	public <T extends Comparable<T>> Graph<T, NullValue, NullValue> simplify(Graph<T, NullValue, NullValue> graph)
+			throws Exception {
+
+		switch (value) {
+			case DIRECTED:
+				graph = graph
+					.run(new org.apache.flink.graph.asm.simple.directed.Simplify<T, NullValue, NullValue>());
+				break;
+			case UNDIRECTED:
+				graph = graph
+					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<T, NullValue, NullValue>(false));
+				break;
+			case UNDIRECTED_CLIP_AND_FLIP:
+				graph = graph
+					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<T, NullValue, NullValue>(true));
+				break;
+		}
+
+		return graph;
+	}
+
+	public String getShortString() {
+		switch (value) {
+			case DIRECTED:
+				return "d";
+			case UNDIRECTED:
+				return "u";
+			case UNDIRECTED_CLIP_AND_FLIP:
+				return "\u0254";
+			default:
+				return "";
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ded25be4/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/SimplifyTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/SimplifyTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/SimplifyTest.java
new file mode 100644
index 0000000..12ae7dc
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/SimplifyTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.graph.drivers.parameter.Simplify.Ordering;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SimplifyTest
+extends ParameterTestBase {
+
+	private Simplify parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new Simplify(owner);
+	}
+
+	@Test
+	public void testWithDirected() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--simplify", "directed"}));
+		Assert.assertEquals(Ordering.DIRECTED, parameter.getValue());
+	}
+
+	@Test
+	public void testWithUndirected() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--simplify", "undirected"}));
+		Assert.assertEquals(Ordering.UNDIRECTED, parameter.getValue());
+	}
+
+	@Test
+	public void testWithNoParameter() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(Ordering.NONE, parameter.getValue());
+	}
+}


[2/6] flink git commit: [FLINK-3695] [gelly] ValueArray types

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayFactory.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayFactory.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayFactory.java
new file mode 100644
index 0000000..b7b6282
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayFactory.java
@@ -0,0 +1,81 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.types.CopyableValue;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.types.Value;
+
+/**
+ * A factory generator for {@link ValueArray} types is necessary because the
+ * contained {@link Value} types do not currently implement a common interface
+ * for creating a {@link ValueArray}. Algorithms must instantiate classes at
+ * runtime when the type information has been erased.
+ *
+ * This mirrors creating {@link Value} using {@link CopyableValue#copy()}.
+ */
+public class ValueArrayFactory {
+
+	/**
+	 * Produce a {@code ValueArray} for the given {@code Value} type.
+	 *
+	 * @param cls {@code Value} class
+	 * @return {@code ValueArray} for given {@code Value} class
+	 */
+	@SuppressWarnings("unchecked")
+	public static <T> ValueArray<T> createValueArray(Class<? extends Value> cls) {
+		if (IntValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new IntValueArray();
+		} else if (LongValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new LongValueArray();
+		} else if (NullValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new NullValueArray();
+		} else if (StringValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new StringValueArray();
+		} else {
+			throw new IllegalArgumentException("Unable to create unbounded ValueArray for type " + cls);
+		}
+	}
+
+	/**
+	 * Produce a {@code ValueArray} for the given {@code Value} type with the
+	 * given bounded size.
+	 *
+	 * @param cls {@code Value} class
+	 * @param bytes limit the array to the given number of bytes
+	 * @return {@code ValueArray} for given {@code Value} class
+	 */
+	@SuppressWarnings("unchecked")
+	public static <T> ValueArray<T> createValueArray(Class<? extends Value> cls, int bytes) {
+		if (IntValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new IntValueArray(bytes);
+		} else if (LongValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new LongValueArray(bytes);
+		} else if (NullValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new NullValueArray(bytes);
+		} else if (StringValue.class.isAssignableFrom(cls)) {
+			return (ValueArray<T>) new StringValueArray(bytes);
+		} else {
+			throw new IllegalArgumentException("Unable to create bounded ValueArray for type " + cls);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfo.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfo.java
new file mode 100644
index 0000000..ee9b770
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfo.java
@@ -0,0 +1,159 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.ValueTypeInfo;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.types.Value;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A {@link TypeInformation} for the {@link ValueArray} type.
+ *
+ * @param <T> the {@link Value} type
+ */
+public class ValueArrayTypeInfo<T> extends TypeInformation<ValueArray<T>> implements AtomicType<ValueArray<T>> {
+
+	private static final long serialVersionUID = 1L;
+
+	public static final ValueArrayTypeInfo<IntValue> INT_VALUE_ARRAY_TYPE_INFO = new ValueArrayTypeInfo<>(ValueTypeInfo.INT_VALUE_TYPE_INFO);
+	public static final ValueArrayTypeInfo<LongValue> LONG_VALUE_ARRAY_TYPE_INFO = new ValueArrayTypeInfo<>(ValueTypeInfo.LONG_VALUE_TYPE_INFO);
+	public static final ValueArrayTypeInfo<NullValue> NULL_VALUE_ARRAY_TYPE_INFO = new ValueArrayTypeInfo<>(ValueTypeInfo.NULL_VALUE_TYPE_INFO);
+	public static final ValueArrayTypeInfo<StringValue> STRING_VALUE_ARRAY_TYPE_INFO = new ValueArrayTypeInfo<>(ValueTypeInfo.STRING_VALUE_TYPE_INFO);
+
+	private final TypeInformation<T> valueType;
+
+	private final Class<T> type;
+
+	public ValueArrayTypeInfo(TypeInformation<T> valueType) {
+		this.valueType = valueType;
+		this.type = valueType.getTypeClass();
+	}
+
+	@Override
+	public int getArity() {
+		return 1;
+	}
+
+	@Override
+	public int getTotalFields() {
+		return 1;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public Class<ValueArray<T>> getTypeClass() {
+		return (Class<ValueArray<T>>) (Class<?>) ValueArray.class;
+	}
+
+	@Override
+	public boolean isBasicType() {
+		return false;
+	}
+
+	@Override
+	public boolean isTupleType() {
+		return false;
+	}
+
+	@Override
+	public boolean isKeyType() {
+		return Comparable.class.isAssignableFrom(type);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public TypeSerializer<ValueArray<T>> createSerializer(ExecutionConfig executionConfig) {
+		if (IntValue.class.isAssignableFrom(type)) {
+			return (TypeSerializer<ValueArray<T>>) (TypeSerializer<?>) new IntValueArraySerializer();
+		} else if (LongValue.class.isAssignableFrom(type)) {
+			return (TypeSerializer<ValueArray<T>>) (TypeSerializer<?>) new LongValueArraySerializer();
+		} else if (NullValue.class.isAssignableFrom(type)) {
+			return (TypeSerializer<ValueArray<T>>) (TypeSerializer<?>) new NullValueArraySerializer();
+		} else if (StringValue.class.isAssignableFrom(type)) {
+			return (TypeSerializer<ValueArray<T>>) (TypeSerializer<?>) new StringValueArraySerializer();
+		} else {
+			throw new InvalidTypesException("No ValueArray class exists for " + type);
+		}
+	}
+
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	@Override
+	public TypeComparator<ValueArray<T>> createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) {
+		if (IntValue.class.isAssignableFrom(type)) {
+			return (TypeComparator<ValueArray<T>>) (TypeComparator<?>) new IntValueArrayComparator(sortOrderAscending);
+		} else if (LongValue.class.isAssignableFrom(type)) {
+			return (TypeComparator<ValueArray<T>>) (TypeComparator<?>) new LongValueArrayComparator(sortOrderAscending);
+		} else if (NullValue.class.isAssignableFrom(type)) {
+			return (TypeComparator<ValueArray<T>>) (TypeComparator<?>) new NullValueArrayComparator(sortOrderAscending);
+		} else if (StringValue.class.isAssignableFrom(type)) {
+			return (TypeComparator<ValueArray<T>>) (TypeComparator<?>) new StringValueArrayComparator(sortOrderAscending);
+		} else {
+			throw new InvalidTypesException("No ValueArray class exists for " + type);
+		}
+	}
+
+	@Override
+	public Map<String, TypeInformation<?>> getGenericParameters() {
+		Map<String, TypeInformation<?>> m = new HashMap<>(1);
+		m.put("T", valueType);
+		return m;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return type.hashCode();
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ValueArrayTypeInfo) {
+			@SuppressWarnings("unchecked")
+			ValueArrayTypeInfo<T> valueArrayTypeInfo = (ValueArrayTypeInfo<T>) obj;
+
+			return valueArrayTypeInfo.canEqual(this) &&
+				type == valueArrayTypeInfo.type;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof ValueArrayTypeInfo;
+	}
+
+	@Override
+	public String toString() {
+		return "ValueArrayType<" + type.getSimpleName() + ">";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoFactory.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoFactory.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoFactory.java
new file mode 100644
index 0000000..2145c3d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoFactory.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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.types.Value;
+
+import java.lang.reflect.Type;
+import java.util.Map;
+
+/**
+ * Used by {@link TypeExtractor} to create a {@link TypeInformation} for
+ * implementations of {@link ValueArray}.
+ *
+ * @param <T> the {@link Value} type
+ */
+public class ValueArrayTypeInfoFactory<T> extends TypeInfoFactory<ValueArray<T>> {
+
+	@Override
+	public TypeInformation<ValueArray<T>> createTypeInfo(Type t, Map<String, TypeInformation<?>> genericParameters) {
+		return new ValueArrayTypeInfo(genericParameters.get("T"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparatorTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparatorTest.java
new file mode 100644
index 0000000..bbc6846
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparatorTest.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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.IntValue;
+
+public class IntValueArrayComparatorTest extends ComparatorTestBase<IntValueArray> {
+
+	@Override
+	protected TypeComparator<IntValueArray> createComparator(boolean ascending) {
+		return new IntValueArrayComparator(ascending);
+	}
+
+	@Override
+	protected TypeSerializer<IntValueArray> createSerializer() {
+		return new IntValueArraySerializer();
+	}
+
+	@Override
+	protected IntValueArray[] getSortedTestData() {
+		IntValueArray iva0 = new IntValueArray();
+
+		IntValueArray iva1 = new IntValueArray();
+		iva1.add(new IntValue(5));
+
+		IntValueArray iva2 = new IntValueArray();
+		iva2.add(new IntValue(5));
+		iva2.add(new IntValue(10));
+
+		return new IntValueArray[]{ iva0, iva1, iva2 };
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java
new file mode 100644
index 0000000..1ee24c9
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.SerializerTestBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.IntValue;
+
+import java.util.Random;
+
+/**
+ * A test for the {@link IntValueArraySerializer}.
+ */
+public class IntValueArraySerializerTest extends SerializerTestBase<IntValueArray> {
+
+	@Override
+	protected TypeSerializer<IntValueArray> createSerializer() {
+		return new IntValueArraySerializer();
+	}
+
+	@Override
+	protected int getLength() {
+		return -1;
+	}
+
+	@Override
+	protected Class<IntValueArray> getTypeClass() {
+		return IntValueArray.class;
+	}
+
+	@Override
+	protected IntValueArray[] getTestData() {
+		int defaultElements = IntValueArray.DEFAULT_CAPACITY_IN_BYTES / IntValueArray.ELEMENT_LENGTH_IN_BYTES;
+
+		Random rnd = new Random(874597969123412341L);
+		int rndInt = rnd.nextInt();
+
+		IntValueArray iva0 = new IntValueArray();
+
+		IntValueArray iva1 = new IntValueArray();
+		iva1.addAll(iva0);
+		iva1.add(new IntValue(0));
+
+		IntValueArray iva2 = new IntValueArray();
+		iva2.addAll(iva1);
+		iva2.add(new IntValue(1));
+
+		IntValueArray iva3 = new IntValueArray();
+		iva3.addAll(iva2);
+		iva3.add(new IntValue(-1));
+
+		IntValueArray iva4 = new IntValueArray();
+		iva4.addAll(iva3);
+		iva4.add(new IntValue(Integer.MAX_VALUE));
+
+		IntValueArray iva5 = new IntValueArray();
+		iva5.addAll(iva4);
+		iva5.add(new IntValue(Integer.MIN_VALUE));
+
+		IntValueArray iva6 = new IntValueArray();
+		iva6.addAll(iva5);
+		iva6.add(new IntValue(rndInt));
+
+		IntValueArray iva7 = new IntValueArray();
+		iva7.addAll(iva6);
+		iva7.add(new IntValue(-rndInt));
+
+		IntValueArray iva8 = new IntValueArray();
+		iva8.addAll(iva7);
+		for (int i = 0 ; i < 1.5 * defaultElements ; i++) {
+			iva8.add(new IntValue(i));
+		}
+		iva8.addAll(iva8);
+
+		return new IntValueArray[] {iva0, iva1, iva2, iva3, iva4, iva5, iva6, iva7, iva8};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayTest.java
new file mode 100644
index 0000000..2e1282a
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArrayTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.types.IntValue;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class IntValueArrayTest {
+
+	@Test
+	public void testBoundedArray() {
+		int count = IntValueArray.DEFAULT_CAPACITY_IN_BYTES / IntValueArray.ELEMENT_LENGTH_IN_BYTES;
+
+		ValueArray<IntValue> iva = new IntValueArray(IntValueArray.DEFAULT_CAPACITY_IN_BYTES);
+
+		// fill the array
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(iva.isFull());
+			assertEquals(i, iva.size());
+
+			assertTrue(iva.add(new IntValue(i)));
+
+			assertEquals(i + 1, iva.size());
+		}
+
+		// array is now full
+		assertTrue(iva.isFull());
+		assertEquals(count, iva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (IntValue lv : iva) {
+			assertEquals(idx++, lv.getValue());
+		}
+
+		// add element past end of array
+		assertFalse(iva.add(new IntValue(count)));
+		assertFalse(iva.addAll(iva));
+
+		// test copy
+		assertEquals(iva, iva.copy());
+
+		// test copyTo
+		IntValueArray iva_to = new IntValueArray();
+		iva.copyTo(iva_to);
+		assertEquals(iva, iva_to);
+
+		// test clear
+		iva.clear();
+		assertEquals(0, iva.size());
+	}
+
+	@Test
+	public void testUnboundedArray() {
+		int count = 4096;
+
+		ValueArray<IntValue> iva = new IntValueArray();
+
+		// add several elements
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(iva.isFull());
+			assertEquals(i, iva.size());
+
+			assertTrue(iva.add(new IntValue(i)));
+
+			assertEquals(i + 1, iva.size());
+		}
+
+		// array never fills
+		assertFalse(iva.isFull());
+		assertEquals(count, iva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (IntValue lv : iva) {
+			assertEquals(idx++, lv.getValue());
+		}
+
+		// add element past end of array
+		assertTrue(iva.add(new IntValue(count)));
+		assertTrue(iva.addAll(iva));
+
+		// test copy
+		assertEquals(iva, iva.copy());
+
+		// test copyTo
+		IntValueArray iva_to = new IntValueArray();
+		iva.copyTo(iva_to);
+		assertEquals(iva, iva_to);
+
+		// test mark/reset
+		int size = iva.size();
+		iva.mark();
+		assertTrue(iva.add(new IntValue()));
+		assertEquals(size + 1, iva.size());
+		iva.reset();
+		assertEquals(size, iva.size());
+
+		// test clear
+		iva.clear();
+		assertEquals(0, iva.size());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparatorTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparatorTest.java
new file mode 100644
index 0000000..af9dbdb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparatorTest.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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.LongValue;
+
+public class LongValueArrayComparatorTest extends ComparatorTestBase<LongValueArray> {
+
+	@Override
+	protected TypeComparator<LongValueArray> createComparator(boolean ascending) {
+		return new LongValueArrayComparator(ascending);
+	}
+
+	@Override
+	protected TypeSerializer<LongValueArray> createSerializer() {
+		return new LongValueArraySerializer();
+	}
+
+	@Override
+	protected LongValueArray[] getSortedTestData() {
+		LongValueArray lva0 = new LongValueArray();
+
+		LongValueArray lva1 = new LongValueArray();
+		lva1.add(new LongValue(5));
+
+		LongValueArray lva2 = new LongValueArray();
+		lva2.add(new LongValue(50));
+		lva2.add(new LongValue(100));
+
+		return new LongValueArray[]{ lva0, lva1, lva2 };
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java
new file mode 100644
index 0000000..1cd0a6c
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.SerializerTestBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.LongValue;
+
+import java.util.Random;
+
+/**
+ * A test for the {@link LongValueArraySerializer}.
+ */
+public class LongValueArraySerializerTest extends SerializerTestBase<LongValueArray> {
+
+	@Override
+	protected TypeSerializer<LongValueArray> createSerializer() {
+		return new LongValueArraySerializer();
+	}
+
+	@Override
+	protected int getLength() {
+		return -1;
+	}
+
+	@Override
+	protected Class<LongValueArray> getTypeClass() {
+		return LongValueArray.class;
+	}
+
+	@Override
+	protected LongValueArray[] getTestData() {
+		int defaultElements = LongValueArray.DEFAULT_CAPACITY_IN_BYTES / LongValueArray.ELEMENT_LENGTH_IN_BYTES;
+
+		Random rnd = new Random(874597969123412341L);
+		long rndLong = rnd.nextLong();
+
+		LongValueArray lva0 = new LongValueArray();
+
+		LongValueArray lva1 = new LongValueArray();
+		lva1.addAll(lva0);
+		lva1.add(new LongValue(0));
+
+		LongValueArray lva2 = new LongValueArray();
+		lva2.addAll(lva1);
+		lva2.add(new LongValue(1));
+
+		LongValueArray lva3 = new LongValueArray();
+		lva3.addAll(lva2);
+		lva3.add(new LongValue(-1));
+
+		LongValueArray lva4 = new LongValueArray();
+		lva4.addAll(lva3);
+		lva4.add(new LongValue(Long.MAX_VALUE));
+
+		LongValueArray lva5 = new LongValueArray();
+		lva5.addAll(lva4);
+		lva5.add(new LongValue(Long.MIN_VALUE));
+
+		LongValueArray lva6 = new LongValueArray();
+		lva6.addAll(lva5);
+		lva6.add(new LongValue(rndLong));
+
+		LongValueArray lva7 = new LongValueArray();
+		lva7.addAll(lva6);
+		lva7.add(new LongValue(-rndLong));
+
+		LongValueArray lva8 = new LongValueArray();
+		lva8.addAll(lva7);
+		for (int i = 0 ; i < 1.5 * defaultElements ; i++) {
+			lva8.add(new LongValue(i));
+		}
+		lva8.addAll(lva8);
+
+		return new LongValueArray[] {lva0, lva1, lva2, lva3, lva4, lva5, lva6, lva7, lva8};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayTest.java
new file mode 100644
index 0000000..cfc345e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArrayTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.types.LongValue;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class LongValueArrayTest {
+
+	@Test
+	public void testBoundedArray() {
+		int count = LongValueArray.DEFAULT_CAPACITY_IN_BYTES / LongValueArray.ELEMENT_LENGTH_IN_BYTES;
+
+		ValueArray<LongValue> lva = new LongValueArray(LongValueArray.DEFAULT_CAPACITY_IN_BYTES);
+
+		// fill the array
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(lva.isFull());
+			assertEquals(i, lva.size());
+
+			assertTrue(lva.add(new LongValue(i)));
+
+			assertEquals(i + 1, lva.size());
+		}
+
+		// array is now full
+		assertTrue(lva.isFull());
+		assertEquals(count, lva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (LongValue lv : lva) {
+			assertEquals(idx++, lv.getValue());
+		}
+
+		// add element past end of array
+		assertFalse(lva.add(new LongValue(count)));
+		assertFalse(lva.addAll(lva));
+
+		// test copy
+		assertEquals(lva, lva.copy());
+
+		// test copyTo
+		LongValueArray lva_to = new LongValueArray();
+		lva.copyTo(lva_to);
+		assertEquals(lva, lva_to);
+
+		// test clear
+		lva.clear();
+		assertEquals(0, lva.size());
+	}
+
+	@Test
+	public void testUnboundedArray() {
+		int count = 4096;
+
+		ValueArray<LongValue> lva = new LongValueArray();
+
+		// add several elements
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(lva.isFull());
+			assertEquals(i, lva.size());
+
+			assertTrue(lva.add(new LongValue(i)));
+
+			assertEquals(i + 1, lva.size());
+		}
+
+		// array never fills
+		assertFalse(lva.isFull());
+		assertEquals(count, lva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (LongValue lv : lva) {
+			assertEquals(idx++, lv.getValue());
+		}
+
+		// add element past end of array
+		assertTrue(lva.add(new LongValue(count)));
+		assertTrue(lva.addAll(lva));
+
+		// test copy
+		assertEquals(lva, lva.copy());
+
+		// test copyTo
+		LongValueArray lva_to = new LongValueArray();
+		lva.copyTo(lva_to);
+		assertEquals(lva, lva_to);
+
+		// test mark/reset
+		int size = lva.size();
+		lva.mark();
+		assertTrue(lva.add(new LongValue()));
+		assertEquals(size + 1, lva.size());
+		lva.reset();
+		assertEquals(size, lva.size());
+
+		// test clear
+		lva.clear();
+		assertEquals(0, lva.size());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparatorTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparatorTest.java
new file mode 100644
index 0000000..10fa2e2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparatorTest.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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.NullValue;
+
+public class NullValueArrayComparatorTest extends ComparatorTestBase<NullValueArray> {
+
+	@Override
+	protected TypeComparator<NullValueArray> createComparator(boolean ascending) {
+		return new NullValueArrayComparator(ascending);
+	}
+
+	@Override
+	protected TypeSerializer<NullValueArray> createSerializer() {
+		return new NullValueArraySerializer();
+	}
+
+	@Override
+	protected NullValueArray[] getSortedTestData() {
+		NullValueArray nva0 = new NullValueArray();
+
+		NullValueArray nva1 = new NullValueArray();
+		nva1.add(NullValue.getInstance());
+
+		NullValueArray nva2 = new NullValueArray();
+		nva2.add(NullValue.getInstance());
+		nva2.add(NullValue.getInstance());
+
+		return new NullValueArray[]{ nva0, nva1 };
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java
new file mode 100644
index 0000000..2253a42
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.SerializerTestBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.NullValue;
+
+/**
+ * A test for the {@link NullValueArraySerializer}.
+ */
+public class NullValueArraySerializerTest extends SerializerTestBase<NullValueArray> {
+
+	@Override
+	protected TypeSerializer<NullValueArray> createSerializer() {
+		return new NullValueArraySerializer();
+	}
+
+	@Override
+	protected int getLength() {
+		return 4;
+	}
+
+	@Override
+	protected Class<NullValueArray> getTypeClass() {
+		return NullValueArray.class;
+	}
+
+	@Override
+	protected NullValueArray[] getTestData() {
+		NullValue nv = NullValue.getInstance();
+
+		NullValueArray nva0 = new NullValueArray();
+
+		NullValueArray nva1 = new NullValueArray();
+		nva1.addAll(nva0);
+		nva1.add(nv);
+
+		NullValueArray nva2 = new NullValueArray();
+		nva2.addAll(nva1);
+		nva2.add(nv);
+
+		NullValueArray nva3 = new NullValueArray();
+		nva3.addAll(nva2);
+		for (int i = 0 ; i < 100 ; i++) {
+			nva3.add(nv);
+		}
+		nva3.addAll(nva3);
+
+		return new NullValueArray[] {nva0, nva1, nva2, nva3};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayTest.java
new file mode 100644
index 0000000..6d013a1
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArrayTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class NullValueArrayTest {
+
+	@Test
+	public void testUnboundedArray() {
+		int count = 4096;
+
+		ValueArray<NullValue> nva = new NullValueArray();
+
+		// add several elements
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(nva.isFull());
+			assertEquals(i, nva.size());
+
+			assertTrue(nva.add(NullValue.getInstance()));
+
+			assertEquals(i + 1, nva.size());
+		}
+
+		// array never fills
+		assertFalse(nva.isFull());
+		assertEquals(count, nva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (NullValue nv : nva) {
+			assertEquals(NullValue.getInstance(), nv);
+		}
+
+		// add element past end of array
+		assertTrue(nva.add(NullValue.getInstance()));
+		assertTrue(nva.addAll(nva));
+
+		// test copy
+		assertEquals(nva, nva.copy());
+
+		// test copyTo
+		NullValueArray nva_to = new NullValueArray();
+		nva.copyTo(nva_to);
+		assertEquals(nva, nva_to);
+
+		// test mark/reset
+		int size = nva.size();
+		nva.mark();
+		assertTrue(nva.add(NullValue.getInstance()));
+		assertEquals(size + 1, nva.size());
+		nva.reset();
+		assertEquals(size, nva.size());
+
+		// test clear
+		nva.clear();
+		assertEquals(0, nva.size());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparatorTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparatorTest.java
new file mode 100644
index 0000000..e7cc102
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparatorTest.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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.ComparatorTestBase;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.StringValue;
+
+public class StringValueArrayComparatorTest extends ComparatorTestBase<StringValueArray> {
+
+	@Override
+	protected TypeComparator<StringValueArray> createComparator(boolean ascending) {
+		return new StringValueArrayComparator(ascending);
+	}
+
+	@Override
+	protected TypeSerializer<StringValueArray> createSerializer() {
+		return new StringValueArraySerializer();
+	}
+
+	@Override
+	protected StringValueArray[] getSortedTestData() {
+		StringValueArray sva0 = new StringValueArray();
+
+		StringValueArray sva1 = new StringValueArray();
+		sva1.add(new StringValue("abc"));
+
+		StringValueArray sva2 = new StringValueArray();
+		sva2.add(new StringValue("qrs"));
+		sva2.add(new StringValue("xyz"));
+
+		return new StringValueArray[]{ sva0, sva1, sva2 };
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java
new file mode 100644
index 0000000..f5909da
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.SerializerTestBase;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.StringValue;
+
+import java.util.Random;
+
+/**
+ * A test for the {@link StringValueArraySerializer}.
+ */
+public class StringValueArraySerializerTest extends SerializerTestBase<StringValueArray> {
+
+	@Override
+	protected TypeSerializer<StringValueArray> createSerializer() {
+		return new StringValueArraySerializer();
+	}
+
+	@Override
+	protected int getLength() {
+		return -1;
+	}
+
+	@Override
+	protected Class<StringValueArray> getTypeClass() {
+		return StringValueArray.class;
+	}
+
+	@Override
+	protected StringValueArray[] getTestData() {
+		int defaultElements = StringValueArray.DEFAULT_CAPACITY_IN_BYTES;
+
+		Random rnd = new Random(874597969123412341L);
+		long rndLong = rnd.nextLong();
+
+		StringValueArray sva0 = new StringValueArray();
+
+		StringValueArray sva1 = new StringValueArray();
+		sva1.addAll(sva0);
+		sva1.add(new StringValue(String.valueOf(0)));
+
+		StringValueArray sva2 = new StringValueArray();
+		sva2.addAll(sva1);
+		sva2.add(new StringValue(String.valueOf(1)));
+
+		StringValueArray sva3 = new StringValueArray();
+		sva3.addAll(sva2);
+		sva3.add(new StringValue(String.valueOf(-1)));
+
+		StringValueArray sva4 = new StringValueArray();
+		sva4.addAll(sva3);
+		sva4.add(new StringValue(String.valueOf(Long.MAX_VALUE)));
+
+		StringValueArray sva5 = new StringValueArray();
+		sva5.addAll(sva4);
+		sva5.add(new StringValue(String.valueOf(Long.MIN_VALUE)));
+
+		StringValueArray sva6 = new StringValueArray();
+		sva6.addAll(sva5);
+		sva6.add(new StringValue(String.valueOf(rndLong)));
+
+		StringValueArray sva7 = new StringValueArray();
+		sva7.addAll(sva6);
+		sva7.add(new StringValue(String.valueOf(-rndLong)));
+
+		StringValueArray sva8 = new StringValueArray();
+		sva8.addAll(sva7);
+		for (int i = 0 ; i < 1.5 * defaultElements ; i++) {
+			sva8.add(new StringValue(String.valueOf(i)));
+		}
+		sva8.addAll(sva8);
+
+		return new StringValueArray[] {sva0, sva1, sva2, sva3, sva4, sva5, sva6, sva7, sva8};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayTest.java
new file mode 100644
index 0000000..a425e8e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArrayTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.types.StringValue;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class StringValueArrayTest {
+
+	@Test
+	public void testBoundedArray() {
+		// one byte for length and one byte for character
+		int count = StringValueArray.DEFAULT_CAPACITY_IN_BYTES / 2;
+
+		ValueArray<StringValue> sva = new StringValueArray(StringValueArray.DEFAULT_CAPACITY_IN_BYTES);
+
+		// fill the array
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(sva.isFull());
+			assertEquals(i, sva.size());
+
+			assertTrue(sva.add(new StringValue(Character.toString((char)(i & 0x7F)))));
+
+			assertEquals(i + 1, sva.size());
+		}
+
+		// array is now full
+		assertTrue(sva.isFull());
+		assertEquals(count, sva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (StringValue sv : sva) {
+			assertEquals((idx++) & 0x7F, sv.getValue().charAt(0));
+		}
+
+		// add element past end of array
+		assertFalse(sva.add(new StringValue(String.valueOf((char)count))));
+		assertFalse(sva.addAll(sva));
+
+		// test copy
+		assertEquals(sva, sva.copy());
+
+		// test copyTo
+		StringValueArray sva_to = new StringValueArray();
+		sva.copyTo(sva_to);
+		assertEquals(sva, sva_to);
+
+		// test clear
+		sva.clear();
+		assertEquals(0, sva.size());
+	}
+
+	@Test
+	public void testBoundedArrayWithVariableLengthCharacters() {
+		// characters alternatingly take 1 and 2 bytes (plus one byte for length)
+		int count = 1280;
+
+		ValueArray<StringValue> sva = new StringValueArray(3200);
+
+		// fill the array
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(sva.isFull());
+			assertEquals(i, sva.size());
+
+			assertTrue(sva.add(new StringValue(Character.toString((char)(i & 0xFF)))));
+
+			assertEquals(i + 1, sva.size());
+		}
+
+		// array is now full
+		assertTrue(sva.isFull());
+		assertEquals(count, sva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (StringValue sv : sva) {
+			assertEquals((idx++) & 0xFF, sv.getValue().charAt(0));
+		}
+
+		// add element past end of array
+		assertFalse(sva.add(new StringValue(String.valueOf((char)count))));
+		assertFalse(sva.addAll(sva));
+
+		// test copy
+		assertEquals(sva, sva.copy());
+
+		// test copyTo
+		StringValueArray sva_to = new StringValueArray();
+		sva.copyTo(sva_to);
+		assertEquals(sva, sva_to);
+
+		// test clear
+		sva.clear();
+		assertEquals(0, sva.size());
+	}
+
+	@Test
+	public void testUnboundedArray() {
+		int count = 4096;
+
+		ValueArray<StringValue> sva = new StringValueArray();
+
+		// add several elements
+		for (int i = 0 ; i < count ; i++) {
+			assertFalse(sva.isFull());
+			assertEquals(i, sva.size());
+
+			assertTrue(sva.add(new StringValue(String.valueOf((char)i))));
+
+			assertEquals(i + 1, sva.size());
+		}
+
+		// array never fills
+		assertFalse(sva.isFull());
+		assertEquals(count, sva.size());
+
+		// verify the array values
+		int idx = 0;
+		for (StringValue sv : sva) {
+			assertEquals(idx++, sv.getValue().charAt(0));
+		}
+
+		// add element past end of array
+		assertTrue(sva.add(new StringValue(String.valueOf((char)count))));
+		assertTrue(sva.addAll(sva));
+
+		// test copy
+		assertEquals(sva, sva.copy());
+
+		// test copyTo
+		StringValueArray sva_to = new StringValueArray();
+		sva.copyTo(sva_to);
+		assertEquals(sva, sva_to);
+
+		// test mark/reset
+		int size = sva.size();
+		sva.mark();
+		assertTrue(sva.add(new StringValue()));
+		assertEquals(size + 1, sva.size());
+		sva.reset();
+		assertEquals(size, sva.size());
+
+		// test clear
+		sva.clear();
+		assertEquals(0, sva.size());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoTest.java
new file mode 100644
index 0000000..73cecc0
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArrayTypeInfoTest.java
@@ -0,0 +1,64 @@
+/*
+ *
+ *  * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.CopyableValueComparator;
+import org.junit.Test;
+
+import static org.apache.flink.graph.types.valuearray.ValueArrayTypeInfo.INT_VALUE_ARRAY_TYPE_INFO;
+import static org.apache.flink.graph.types.valuearray.ValueArrayTypeInfo.LONG_VALUE_ARRAY_TYPE_INFO;
+import static org.apache.flink.graph.types.valuearray.ValueArrayTypeInfo.NULL_VALUE_ARRAY_TYPE_INFO;
+import static org.apache.flink.graph.types.valuearray.ValueArrayTypeInfo.STRING_VALUE_ARRAY_TYPE_INFO;
+import static org.junit.Assert.assertEquals;
+
+public class ValueArrayTypeInfoTest {
+
+	private ExecutionConfig config = new ExecutionConfig();
+
+	@Test
+	public void testIntValueArray() {
+		assertEquals(INT_VALUE_ARRAY_TYPE_INFO.getTypeClass(), ValueArray.class);
+		assertEquals(INT_VALUE_ARRAY_TYPE_INFO.createSerializer(config).getClass(), IntValueArraySerializer.class);
+		assertEquals(INT_VALUE_ARRAY_TYPE_INFO.createComparator(true, config).getClass(), IntValueArrayComparator.class);
+	}
+
+	@Test
+	public void testLongValueArray() {
+		assertEquals(LONG_VALUE_ARRAY_TYPE_INFO.getTypeClass(), ValueArray.class);
+		assertEquals(LONG_VALUE_ARRAY_TYPE_INFO.createSerializer(config).getClass(), LongValueArraySerializer.class);
+		assertEquals(LONG_VALUE_ARRAY_TYPE_INFO.createComparator(true, config).getClass(), LongValueArrayComparator.class);
+	}
+
+	@Test
+	public void testNullValueArray() {
+		assertEquals(NULL_VALUE_ARRAY_TYPE_INFO.getTypeClass(), ValueArray.class);
+		assertEquals(NULL_VALUE_ARRAY_TYPE_INFO.createSerializer(config).getClass(), NullValueArraySerializer.class);
+		assertEquals(NULL_VALUE_ARRAY_TYPE_INFO.createComparator(true, config).getClass(), NullValueArrayComparator.class);
+	}
+
+	@Test
+	public void testStringValueArray() {
+		assertEquals(STRING_VALUE_ARRAY_TYPE_INFO.getTypeClass(), ValueArray.class);
+		assertEquals(STRING_VALUE_ARRAY_TYPE_INFO.createSerializer(config).getClass(), StringValueArraySerializer.class);
+		assertEquals(STRING_VALUE_ARRAY_TYPE_INFO.createComparator(true, config).getClass(), StringValueArrayComparator.class);
+	}
+}


[5/6] flink git commit: [FLINK-5913] [gelly] Example drivers

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/TriangleListing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/TriangleListing.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/TriangleListing.java
index 93a96c4..ca0c167 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/TriangleListing.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/TriangleListing.java
@@ -18,315 +18,125 @@
 
 package org.apache.flink.graph.drivers;
 
-import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAnalytic;
-import org.apache.flink.graph.GraphCsvReader;
-import org.apache.flink.graph.asm.translate.TranslateGraphIds;
-import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
-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.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
-
-import java.text.NumberFormat;
+import org.apache.flink.graph.asm.result.PrintableResult;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.BooleanParameter;
+import org.apache.flink.graph.drivers.parameter.ChoiceParameter;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.types.CopyableValue;
 
 import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
 
 /**
- * Driver for the library implementation of Triangle Listing.
- *
- * This example reads a simple directed or undirected graph from a CSV file or
- * generates an RMat graph with the given scale and edge factor then lists
- * all triangles.
+ * Driver for directed and undirected triangle listing algorithm and analytic.
  *
  * @see org.apache.flink.graph.library.clustering.directed.TriangleListing
+ * @see org.apache.flink.graph.library.clustering.directed.TriadicCensus
  * @see org.apache.flink.graph.library.clustering.undirected.TriangleListing
+ * @see org.apache.flink.graph.library.clustering.undirected.TriadicCensus
  */
-public class TriangleListing {
-
-	private static final int DEFAULT_SCALE = 10;
-
-	private static final int DEFAULT_EDGE_FACTOR = 16;
+public class TriangleListing<K extends Comparable<K> & CopyableValue<K>, VV, EV>
+extends SimpleDriver<PrintableResult>
+implements Driver<K, VV, EV>, CSV, Hash, Print {
 
-	private static final boolean DEFAULT_TRIADIC_CENSUS = true;
-
-	private static final boolean DEFAULT_CLIP_AND_FLIP = true;
-
-	private static String getUsage(String message) {
-		return new StrBuilder()
-			.appendNewLine()
-			.appendln(WordUtils.wrap("Lists all triangles in a graph.", 80))
-			.appendNewLine()
-			.appendln(WordUtils.wrap("This algorithm returns tuples containing the vertex IDs for each triangle and" +
-				" for directed graphs a bitmask indicating the presence of the six potential connecting edges.", 80))
-			.appendNewLine()
-			.appendln("usage: TriangleListing --directed <true | false> [--triadic_census <true | false>] --input <csv | rmat> --output <print | hash | csv>")
-			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --input csv --type <integer | string> [--simplify <true | false>] --input_filename FILENAME [--input_line_delimiter LINE_DELIMITER] [--input_field_delimiter FIELD_DELIMITER]")
-			.appendln("  --input rmat [--scale SCALE] [--edge_factor EDGE_FACTOR]")
-			.appendNewLine()
-			.appendln("  --output print")
-			.appendln("  --output hash")
-			.appendln("  --output csv --output_filename FILENAME [--output_line_delimiter LINE_DELIMITER] [--output_field_delimiter FIELD_DELIMITER]")
-			.appendNewLine()
-			.appendln("Usage error: " + message)
-			.toString();
-	}
+	private static final String DIRECTED = "directed";
 
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
+	private static final String UNDIRECTED = "undirected";
 
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
+	private ChoiceParameter order = new ChoiceParameter(this, "order")
+		.addChoices(DIRECTED, UNDIRECTED);
 
-		if (! parameters.has("directed")) {
-			throw new ProgramParametrizationException(getUsage("must declare execution mode as '--directed true' or '--directed false'"));
-		}
-		boolean directedAlgorithm = parameters.getBoolean("directed");
+	private BooleanParameter sortTriangleVertices = new BooleanParameter(this, "sort_triangle_vertices");
 
-		int little_parallelism = parameters.getInt("little_parallelism", PARALLELISM_DEFAULT);
-		boolean triadic_census = parameters.getBoolean("triadic_census", DEFAULT_TRIADIC_CENSUS);
+	private BooleanParameter computeTriadicCensus = new BooleanParameter(this, "triadic_census");
 
-		GraphAnalytic tc = null;
-		DataSet tl;
-
-		switch (parameters.get("input", "")) {
-			case "csv": {
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				GraphCsvReader reader = Graph
-					.fromCsvReader(parameters.getRequired("input_filename"), env)
-						.ignoreCommentsEdges("#")
-						.lineDelimiterEdges(lineDelimiter)
-						.fieldDelimiterEdges(fieldDelimiter);
-
-				switch (parameters.get("type", "")) {
-					case "integer": {
-						Graph<LongValue, NullValue, NullValue> graph = reader
-							.keyType(LongValue.class);
-
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>()
-										.setParallelism(little_parallelism));
-							}
-
-							if (triadic_census) {
-								tc = graph
-									.run(new org.apache.flink.graph.library.clustering.directed.TriadicCensus<LongValue, NullValue, NullValue>()
-										.setLittleParallelism(little_parallelism));
-							}
-							tl = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.TriangleListing<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(false)
-										.setParallelism(little_parallelism));
-							}
-
-							if (triadic_census) {
-								tc = graph
-									.run(new org.apache.flink.graph.library.clustering.undirected.TriadicCensus<LongValue, NullValue, NullValue>()
-										.setLittleParallelism(little_parallelism));
-							}
-							tl = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-					} break;
-
-					case "string": {
-						Graph<StringValue, NullValue, NullValue> graph = reader
-							.keyType(StringValue.class);
-
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<StringValue, NullValue, NullValue>()
-										.setParallelism(little_parallelism));
-							}
-
-							if (triadic_census) {
-								tc = graph
-									.run(new org.apache.flink.graph.library.clustering.directed.TriadicCensus<StringValue, NullValue, NullValue>()
-										.setLittleParallelism(little_parallelism));
-							}
-							tl = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.TriangleListing<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<StringValue, NullValue, NullValue>(false)
-										.setParallelism(little_parallelism));
-							}
-
-							if (triadic_census) {
-								tc = graph
-									.run(new org.apache.flink.graph.library.clustering.undirected.TriadicCensus<StringValue, NullValue, NullValue>()
-										.setLittleParallelism(little_parallelism));
-							}
-							tl = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-					} break;
-
-					default:
-						throw new ProgramParametrizationException(getUsage("invalid CSV type"));
-				}
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
 
+	private GraphAnalytic<K, VV, EV, ? extends PrintableResult> triadicCensus;
 
-			} break;
-
-			case "rmat": {
-				int scale = parameters.getInt("scale", DEFAULT_SCALE);
-				int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-				RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-				long vertexCount = 1L << scale;
-				long edgeCount = vertexCount * edgeFactor;
-
-				Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-					.generate();
-
-				if (directedAlgorithm) {
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> simpleGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>()
-								.setParallelism(little_parallelism));
-
-						if (triadic_census) {
-							tc = simpleGraph
-								.run(new org.apache.flink.graph.library.clustering.directed.TriadicCensus<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-						tl = simpleGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.TriangleListing<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					} else {
-						Graph<LongValue, NullValue, NullValue> simpleGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>()
-								.setParallelism(little_parallelism));
-
-						if (triadic_census) {
-							tc = simpleGraph
-								.run(new org.apache.flink.graph.library.clustering.directed.TriadicCensus<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-						tl = simpleGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.TriangleListing<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					}
-				} else {
-					boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP);
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
 
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> simpleGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(clipAndFlip)
-								.setParallelism(little_parallelism));
+	@Override
+	public String getShortDescription() {
+		return "list triangles";
+	}
 
-						if (triadic_census) {
-							tc = simpleGraph
-								.run(new org.apache.flink.graph.library.clustering.undirected.TriadicCensus<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-						tl = simpleGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					} else {
-						Graph<IntValue, NullValue, NullValue> simpleGraph = graph
-							.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue())
-								.setParallelism(little_parallelism))
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<IntValue, NullValue, NullValue>(clipAndFlip)
-								.setParallelism(little_parallelism));
+	@Override
+	public String getLongDescription() {
+		return WordUtils.wrap(new StrBuilder()
+			.appendln("List all triangles graph.")
+			.appendNewLine()
+			.append("The algorithm result contains three vertex IDs. For the directed algorithm " +
+				"the result contains an additional bitmask indicating the presence of the six " +
+				"potential connecting edges.")
+			.toString(), 80);
+	}
 
-						if (triadic_census) {
-							tc = simpleGraph
-								.run(new org.apache.flink.graph.library.clustering.undirected.TriadicCensus<IntValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-						tl = simpleGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing<IntValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					}
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		int lp = littleParallelism.getValue().intValue();
+
+		switch (order.getValue()) {
+			case DIRECTED:
+				result = graph
+					.run(new org.apache.flink.graph.library.clustering.directed.TriangleListing<K, VV, EV>()
+						.setSortTriangleVertices(sortTriangleVertices.getValue())
+						.setLittleParallelism(lp));
+
+				if (computeTriadicCensus.getValue()) {
+					triadicCensus = graph
+						.run(new org.apache.flink.graph.library.clustering.directed.TriadicCensus<K, VV, EV>()
+							.setLittleParallelism(lp));
 				}
-			} break;
+				break;
 
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid input type"));
-		}
+			case UNDIRECTED:
+				result = graph
+					.run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing<K, VV, EV>()
+						.setSortTriangleVertices(sortTriangleVertices.getValue())
+						.setLittleParallelism(lp));
 
-		switch (parameters.get("output", "")) {
-			case "print":
-				System.out.println();
-				if (directedAlgorithm) {
-					for (Object e: tl.collect()) {
-						org.apache.flink.graph.library.clustering.directed.TriangleListing.Result result =
-							(org.apache.flink.graph.library.clustering.directed.TriangleListing.Result) e;
-						System.out.println(result.toPrintableString());
-					}
-				} else {
-					tl.print();
+				if (computeTriadicCensus.getValue()) {
+					triadicCensus = graph
+						.run(new org.apache.flink.graph.library.clustering.undirected.TriadicCensus<K, VV, EV>()
+							.setLittleParallelism(lp));
 				}
 				break;
+		}
+	}
 
-			case "hash":
-				System.out.println();
-				System.out.println(DataSetUtils.checksumHashCode(tl));
-				break;
-
-			case "csv":
-				String filename = parameters.getRequired("output_filename");
-
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
+	@Override
+	public void hash(String executionName) throws Exception {
+		super.hash(executionName);
+		printAnalytics();
+	}
 
-				tl.writeAsCsv(filename, lineDelimiter, fieldDelimiter);
+	@Override
+	public void print(String executionName) throws Exception {
+		super.print(executionName);
+		printAnalytics();
+	}
 
-				env.execute();
-				break;
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid output type"));
-		}
+	@Override
+	public void writeCSV(String filename, String lineDelimiter, String fieldDelimiter) {
+		super.writeCSV(filename, lineDelimiter, fieldDelimiter);
+		printAnalytics();
+	}
 
-		if (tc != null) {
+	private void printAnalytics() {
+		if (computeTriadicCensus.getValue()) {
 			System.out.print("Triadic census:\n  ");
-			System.out.println(tc.getResult().toString().replace(";", "\n "));
+			System.out.println(triadicCensus.getResult().toPrintableString().replace(";", "\n "));
 		}
-
-		JobExecutionResult result = env.getLastJobExecutionResult();
-
-		NumberFormat nf = NumberFormat.getInstance();
-		System.out.println();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/IterationConvergence.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/IterationConvergence.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/IterationConvergence.java
new file mode 100644
index 0000000..e9d648a
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/parameter/IterationConvergence.java
@@ -0,0 +1,89 @@
+/*
+ * 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.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.graph.drivers.parameter.IterationConvergence.Value;
+
+/**
+ * Iterative algorithms which converge can be terminated with a maximum number
+ * of iterations or a convergence threshold which stops computation when the
+ * total change in scores is below a given delta.
+ *
+ * If the command-line configuration specifies neither a number of iterations
+ * nor a convergence threshold then a default number of iterations is used
+ * with an infinite convergence threshold. Otherwise, when either value is
+ * configured then an unset value is set to infinity.
+ */
+public class IterationConvergence
+implements Parameter<Value> {
+
+	private final int defaultIterations;
+
+	private final Value value = new Value();
+
+	/**
+	 * Add this parameter to the list of parameters stored by owner.
+	 *
+	 * @param owner the {@link Parameterized} using this {@link Parameter}
+	 * @param defaultIterations the default number of iterations if neither
+	 *                          the number of iterations nor the convergence
+	 *                          threshold are specified
+	 */
+	public IterationConvergence(ParameterizedBase owner, int defaultIterations) {
+		owner.addParameter(this);
+		this.defaultIterations = defaultIterations;
+	}
+
+	@Override
+	public String getUsage() {
+		return "[--iterations ITERATIONS] [--convergence_threshold CONVERGENCE_THRESHOLD]";
+	}
+
+	@Override
+	public void configure(ParameterTool parameterTool) {
+		if (!parameterTool.has("iterations") && !parameterTool.has("convergence_threshold")) {
+			// no configuration so use default iterations and maximum threshold
+			value.iterations = defaultIterations;
+			value.convergenceThreshold = Double.MAX_VALUE;
+		} else {
+			// use configured values and maximum default for unset values
+			value.iterations = parameterTool.getInt("iterations", Integer.MAX_VALUE);
+			Util.checkParameter(value.iterations > 0,
+				"iterations must be greater than zero");
+
+			value.convergenceThreshold = parameterTool.getDouble("convergence_threshold", Double.MAX_VALUE);
+			Util.checkParameter(value.convergenceThreshold > 0,
+				"convergence threshold must be greater than zero");
+		}
+	}
+
+	@Override
+	public Value getValue() {
+		return value;
+	}
+
+	/**
+	 * Encapsulate the number of iterations and the convergence threshold.
+	 */
+	public static class Value {
+		public int iterations;
+		public double convergenceThreshold;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/ConnectedComponents.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/ConnectedComponents.java
deleted file mode 100644
index 6651739..0000000
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/ConnectedComponents.java
+++ /dev/null
@@ -1,141 +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.flink.graph.examples;
-
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.examples.data.ConnectedComponentsDefaultData;
-import org.apache.flink.graph.library.GSAConnectedComponents;
-import org.apache.flink.types.NullValue;
-
-/**
- * This example shows how to use Gelly's library methods.
- * You can find all available library methods in {@link org.apache.flink.graph.library}. 
- * 
- * In particular, this example uses the {@link GSAConnectedComponents}
- * library method to compute the connected components of the input graph.
- *
- * The input file is a plain text file and must be formatted as follows:
- * Edges are represented by tuples of srcVertexId, trgVertexId which are
- * separated by tabs. Edges themselves are separated by newlines.
- * For example: <code>1\t2\n1\t3\n</code> defines two edges,
- * 1-2 with and 1-3.
- *
- * Usage <code>ConnectedComponents &lt;edge path&gt; &lt;result path&gt;
- * &lt;number of iterations&gt; </code><br>
- * If no parameters are provided, the program is run with default data from
- * {@link ConnectedComponentsDefaultData}
- */
-public class ConnectedComponents implements ProgramDescription {
-
-	@SuppressWarnings("serial")
-	public static void main(String [] args) throws Exception {
-
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Edge<Long, NullValue>> edges = getEdgesDataSet(env);
-
-		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new MapFunction<Long, Long>() {
-			@Override
-			public Long map(Long value) throws Exception {
-				return value;
-			}
-		}, env);
-
-		DataSet<Vertex<Long, Long>> verticesWithMinIds = graph
-				.run(new GSAConnectedComponents<Long, Long, NullValue>(maxIterations));
-
-		// emit result
-		if (fileOutput) {
-			verticesWithMinIds.writeAsCsv(outputPath, "\n", ",");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute("Connected Components Example");
-		} else {
-			verticesWithMinIds.print();
-		}
-	}
-
-	@Override
-	public String getDescription() {
-		return "Connected Components Example";
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String edgeInputPath = null;
-	private static String outputPath = null;
-	private static Integer maxIterations = ConnectedComponentsDefaultData.MAX_ITERATIONS;
-
-	private static boolean parseParameters(String [] args) {
-		if(args.length > 0) {
-			if(args.length != 3) {
-				System.err.println("Usage ConnectedComponents <edge path> <output path> " +
-						"<num iterations>");
-				return false;
-			}
-
-			fileOutput = true;
-			edgeInputPath = args[0];
-			outputPath = args[1];
-			maxIterations = Integer.parseInt(args[2]);
-
-		} else {
-			System.out.println("Executing ConnectedComponents example with default parameters and built-in default data.");
-			System.out.println("Provide parameters to read input data from files.");
-			System.out.println("Usage ConnectedComponents <edge path> <output path> " +
-					"<num iterations>");
-		}
-
-		return true;
-	}
-
-	@SuppressWarnings("serial")
-	private static DataSet<Edge<Long, NullValue>> getEdgesDataSet(ExecutionEnvironment env) {
-
-		if(fileOutput) {
-			return env.readCsvFile(edgeInputPath)
-					.ignoreComments("#")
-					.fieldDelimiter("\t")
-					.lineDelimiter("\n")
-					.types(Long.class, Long.class)
-					.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
-						@Override
-						public Edge<Long, NullValue> map(Tuple2<Long, Long> value) throws Exception {
-							return new Edge<>(value.f0, value.f1, NullValue.getInstance());
-						}
-					});
-		} else {
-			return ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
index 35f07b0..1cd3549 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/GSASingleSourceShortestPaths.java
@@ -114,7 +114,7 @@ public class GSASingleSourceShortestPaths implements ProgramDescription {
 		public Double gather(Neighbor<Double, Double> neighbor) {
 			return neighbor.getNeighborValue() + neighbor.getEdgeValue();
 		}
-	};
+	}
 
 	@SuppressWarnings("serial")
 	private static final class ChooseMinDistance extends SumFunction<Double, Double, Double> {
@@ -122,7 +122,7 @@ public class GSASingleSourceShortestPaths implements ProgramDescription {
 		public Double sum(Double newValue, Double currentValue) {
 			return Math.min(newValue, currentValue);
 		}
-	};
+	}
 
 	@SuppressWarnings("serial")
 	private static final class UpdateDistance extends ApplyFunction<Long, Double, Double> {

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/IterationConvergenceTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/IterationConvergenceTest.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/IterationConvergenceTest.java
new file mode 100644
index 0000000..ae92943
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/drivers/parameter/IterationConvergenceTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.graph.drivers.parameter;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IterationConvergenceTest
+extends ParameterTestBase {
+
+	private IterationConvergence parameter;
+
+	@Before
+	public void setup() {
+		super.setup();
+
+		parameter = new IterationConvergence(owner, 10);
+	}
+
+	@Test
+	public void testWithIterations() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--iterations", "42"}));
+		Assert.assertEquals(42, parameter.getValue().iterations);
+		Assert.assertEquals(Double.MAX_VALUE, parameter.getValue().convergenceThreshold, 0.000001);
+	}
+
+	@Test
+	public void testWithConvergenceThreshold() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--convergence_threshold", "42"}));
+		Assert.assertEquals(Integer.MAX_VALUE, parameter.getValue().iterations);
+		Assert.assertEquals(42.0, parameter.getValue().convergenceThreshold, 0.000001);
+	}
+
+	@Test
+	public void testWithBoth() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{"--iterations", "42", "--convergence_threshold", "42"}));
+		Assert.assertEquals(42, parameter.getValue().iterations);
+		Assert.assertEquals(42.0, parameter.getValue().convergenceThreshold, 0.000001);
+	}
+
+	@Test
+	public void testWithNeither() {
+		parameter.configure(ParameterTool.fromArgs(new String[]{}));
+		Assert.assertEquals(10, parameter.getValue().iterations);
+		Assert.assertEquals(Double.MAX_VALUE, parameter.getValue().convergenceThreshold, 0.000001);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/ConnectedComponentsITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/ConnectedComponentsITCase.java
deleted file mode 100644
index d0de8dc..0000000
--- a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/test/examples/ConnectedComponentsITCase.java
+++ /dev/null
@@ -1,72 +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.flink.graph.test.examples;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.graph.examples.ConnectedComponents;
-import org.apache.flink.graph.examples.data.ConnectedComponentsDefaultData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-
-@RunWith(Parameterized.class)
-public class ConnectedComponentsITCase extends MultipleProgramsTestBase {
-
-	private String edgesPath;
-
-	private String resultPath;
-
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public ConnectedComponentsITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		resultPath = tempFolder.newFile().toURI().toString();
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(ConnectedComponentsDefaultData.EDGES, edgesFile, Charsets.UTF_8);
-		edgesPath = edgesFile.toURI().toString();
-	}
-
-	@Test
-	public void testConnectedComponentsExample() throws Exception {
-		ConnectedComponents.main(new String[]{edgesPath, resultPath, ConnectedComponentsDefaultData.MAX_ITERATIONS + ""});
-		expected = ConnectedComponentsDefaultData.VERTICES_WITH_MIN_ID;
-	}
-
-	@After
-	public void after() throws Exception {
-		TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
index cbbfb02..71baaa9 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
@@ -529,7 +529,13 @@ public class Graph<K, VV, EV> {
 
 		TypeInformation<K> keyType = ((TupleTypeInfo<?>) vertices.getType()).getTypeAt(0);
 
-		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, vertices.getType(), null);
+		TypeInformation<NV> valueType;
+
+		if (mapper instanceof ResultTypeQueryable) {
+			valueType = ((ResultTypeQueryable) mapper).getProducedType();
+		} else {
+			valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, vertices.getType(), null);
+		}
 
 		TypeInformation<Vertex<K, NV>> returnType = (TypeInformation<Vertex<K, NV>>) new TupleTypeInfo(
 				Vertex.class, keyType, valueType);
@@ -573,7 +579,13 @@ public class Graph<K, VV, EV> {
 
 		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
 
-		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, edges.getType(), null);
+		TypeInformation<NV> valueType;
+
+		if (mapper instanceof ResultTypeQueryable) {
+			valueType = ((ResultTypeQueryable) mapper).getProducedType();
+		} else {
+			valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, edges.getType(), null);
+		}
 
 		TypeInformation<Edge<K, NV>> returnType = (TypeInformation<Edge<K, NV>>) new TupleTypeInfo(
 				Edge.class, keyType, keyType, valueType);

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
index 3cd8f05..959b816 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
@@ -22,13 +22,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.spargel.GatherFunction;
 import org.apache.flink.graph.spargel.MessageIterator;
 import org.apache.flink.graph.spargel.ScatterFunction;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.graph.utils.GraphUtils.MapTo;
 import org.apache.flink.types.NullValue;
 
 /**
@@ -72,7 +73,7 @@ public class ConnectedComponents<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueTypeInfo = ((TupleTypeInfo<?>) graph.getVertices().getType()).getTypeAt(1);
 
 		Graph<K, VV, NullValue> undirectedGraph = graph
-			.mapEdges(new NullValueEdgeMapper<K, EV>())
+			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
 			.getUndirected();
 
 		return undirectedGraph.runScatterGatherIteration(

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
index 327de73..1680f38 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
@@ -29,7 +30,7 @@ import org.apache.flink.graph.gsa.ApplyFunction;
 import org.apache.flink.graph.gsa.GatherFunction;
 import org.apache.flink.graph.gsa.Neighbor;
 import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.graph.utils.GraphUtils.MapTo;
 import org.apache.flink.types.NullValue;
 
 /**
@@ -73,7 +74,7 @@ public class GSAConnectedComponents<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueTypeInfo = ((TupleTypeInfo<?>) graph.getVertices().getType()).getTypeAt(1);
 
 		Graph<K, VV, NullValue> undirectedGraph = graph
-			.mapEdges(new NullValueEdgeMapper<K, EV>())
+			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
 			.getUndirected();
 
 		return undirectedGraph.runGatherSumApplyIteration(
@@ -87,7 +88,6 @@ public class GSAConnectedComponents<K, VV extends Comparable<VV>, EV>
 	//  Connected Components UDFs
 	// --------------------------------------------------------------------------------------------
 
-	@SuppressWarnings("serial")
 	private static final class GatherNeighborIds<VV extends Comparable<VV>>
 		extends GatherFunction<VV, NullValue, VV>
 		implements ResultTypeQueryable<VV> {
@@ -108,7 +108,6 @@ public class GSAConnectedComponents<K, VV extends Comparable<VV>, EV>
 		}
 	}
 
-	@SuppressWarnings("serial")
 	private static final class SelectMinId<VV extends Comparable<VV>>
 		extends SumFunction<VV, NullValue, VV>
 		implements ResultTypeQueryable<VV> {
@@ -129,7 +128,6 @@ public class GSAConnectedComponents<K, VV extends Comparable<VV>, EV>
 		}
 	}
 
-	@SuppressWarnings("serial")
 	private static final class UpdateComponentId<K, VV extends Comparable<VV>>
 		extends ApplyFunction<K, VV, VV>
 		implements ResultTypeQueryable<VV> {

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
index 96e5afc..0064a68 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
@@ -22,13 +22,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAlgorithm;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.spargel.GatherFunction;
 import org.apache.flink.graph.spargel.MessageIterator;
 import org.apache.flink.graph.spargel.ScatterFunction;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.graph.utils.GraphUtils.MapTo;
 import org.apache.flink.types.NullValue;
 
 import java.util.HashMap;
@@ -76,7 +77,7 @@ public class LabelPropagation<K, VV extends Comparable<VV>, EV>
 		TypeInformation<VV> valueType = ((TupleTypeInfo<?>) input.getVertices().getType()).getTypeAt(1);
 		// iteratively adopt the most frequent label among the neighbors of each vertex
 		return input
-			.mapEdges(new NullValueEdgeMapper<K, EV>())
+			.mapEdges(new MapTo<Edge<K, EV>, NullValue>(NullValue.getInstance()))
 			.runScatterGatherIteration(
 				new SendNewLabelToNeighbors<K, VV>(valueType), new UpdateVertexLabel<K, VV>(), maxIterations)
 			.getVertices();

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
index 236272f..6fe753a 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/directed/TriangleListing.java
@@ -35,9 +35,9 @@ import org.apache.flink.graph.EdgeOrder;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.asm.degree.annotate.directed.EdgeDegreesPair;
 import org.apache.flink.graph.asm.degree.annotate.directed.VertexDegrees.Degrees;
-import org.apache.flink.graph.library.clustering.directed.TriangleListing.Result;
 import org.apache.flink.graph.asm.result.PrintableResult;
 import org.apache.flink.graph.asm.result.TertiaryResult;
+import org.apache.flink.graph.library.clustering.directed.TriangleListing.Result;
 import org.apache.flink.graph.utils.proxy.GraphAlgorithmWrappingDataSet;
 import org.apache.flink.graph.utils.proxy.OptionalBoolean;
 import org.apache.flink.types.ByteValue;

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
index 9aca8a4..eda5c1c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java
@@ -32,8 +32,8 @@ import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.asm.degree.annotate.undirected.VertexDegree;
 import org.apache.flink.graph.asm.result.PrintableResult;
-import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result;
 import org.apache.flink.graph.asm.result.UnaryResult;
+import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result;
 import org.apache.flink.graph.utils.Murmur3_32;
 import org.apache.flink.graph.utils.proxy.GraphAlgorithmWrappingDataSet;
 import org.apache.flink.graph.utils.proxy.OptionalBoolean;

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/link_analysis/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/link_analysis/PageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/link_analysis/PageRank.java
index 57743e8..1dfa3ee 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/link_analysis/PageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/link_analysis/PageRank.java
@@ -40,6 +40,8 @@ import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.asm.degree.annotate.directed.EdgeSourceDegrees;
 import org.apache.flink.graph.asm.degree.annotate.directed.VertexDegrees;
 import org.apache.flink.graph.asm.degree.annotate.directed.VertexDegrees.Degrees;
+import org.apache.flink.graph.asm.result.PrintableResult;
+import org.apache.flink.graph.asm.result.UnaryResult;
 import org.apache.flink.graph.library.link_analysis.Functions.SumScore;
 import org.apache.flink.graph.library.link_analysis.PageRank.Result;
 import org.apache.flink.graph.utils.GraphUtils;
@@ -500,7 +502,8 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 	 * @param <T> ID type
 	 */
 	public static class Result<T>
-	extends Tuple2<T, DoubleValue> {
+	extends Tuple2<T, DoubleValue>
+	implements PrintableResult, UnaryResult<T> {
 		public static final int HASH_SEED = 0x4010af29;
 
 		private Murmur3_32 hasher = new Murmur3_32(HASH_SEED);
@@ -518,7 +521,8 @@ extends GraphAlgorithmWrappingDataSet<K, VV, EV, Result<K>> {
 			return f1;
 		}
 
-		public String toVerboseString() {
+		@Override
+		public String toPrintableString() {
 			return "Vertex ID: " + getVertexId0()
 				+ ", PageRank score: " + getPageRankScore();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
index 7d77541..6aaf9f2 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/AdamicAdar.java
@@ -34,8 +34,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.asm.degree.annotate.undirected.VertexDegree;
-import org.apache.flink.graph.asm.result.PrintableResult;
 import org.apache.flink.graph.asm.result.BinaryResult;
+import org.apache.flink.graph.asm.result.PrintableResult;
 import org.apache.flink.graph.library.similarity.AdamicAdar.Result;
 import org.apache.flink.graph.utils.Murmur3_32;
 import org.apache.flink.graph.utils.proxy.GraphAlgorithmWrappingDataSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
index 3b36715..0c80e6d 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/similarity/JaccardIndex.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.java.tuple.Tuple4;
 import org.apache.flink.graph.Edge;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.asm.degree.annotate.undirected.EdgeTargetDegree;
-import org.apache.flink.graph.asm.result.PrintableResult;
 import org.apache.flink.graph.asm.result.BinaryResult;
+import org.apache.flink.graph.asm.result.PrintableResult;
 import org.apache.flink.graph.library.similarity.JaccardIndex.Result;
 import org.apache.flink.graph.utils.Murmur3_32;
 import org.apache.flink.graph.utils.proxy.GraphAlgorithmWrappingDataSet;

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
index 2e0dffc..78fb378 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/GraphUtils.java
@@ -20,7 +20,10 @@ package org.apache.flink.graph.utils;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.types.LongValue;
 
 import static org.apache.flink.api.java.typeutils.ValueTypeInfo.LONG_VALUE_TYPE_INFO;
@@ -62,7 +65,7 @@ public class GraphUtils {
 	 * @param <O> output type
 	 */
 	public static class MapTo<I, O>
-	implements MapFunction<I, O> {
+	implements MapFunction<I, O>, ResultTypeQueryable<O> {
 		private final O value;
 
 		/**
@@ -78,6 +81,11 @@ public class GraphUtils {
 		public O map(I o) throws Exception {
 			return value;
 		}
+
+		@Override
+		public TypeInformation<O> getProducedType() {
+			return (TypeInformation<O>)TypeExtractor.createTypeInfo(value.getClass());
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
deleted file mode 100644
index 2bd4719..0000000
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
+++ /dev/null
@@ -1,32 +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.flink.graph.utils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.types.NullValue;
-
-public class NullValueEdgeMapper<K, EV> implements	MapFunction<Edge<K, EV>, NullValue> {
-
-	private static final long serialVersionUID = 1L;
-
-	public NullValue map(Edge<K, EV> edge) {
-		return NullValue.getInstance();
-	}
-}


[6/6] flink git commit: [FLINK-5913] [gelly] Example drivers

Posted by gr...@apache.org.
[FLINK-5913] [gelly] Example drivers

Replace existing and create new algorithm Driver implementations for
each of the library methods.

This closes #3635


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

Branch: refs/heads/master
Commit: a48357db8c4187fd08f3b17880899ebbcb5d3b5e
Parents: ded25be
Author: Greg Hogan <co...@greghogan.com>
Authored: Wed Oct 26 15:18:50 2016 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 31 11:17:26 2017 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/flink/graph/Usage.java |   2 -
 .../apache/flink/graph/drivers/AdamicAdar.java  |  71 ++++
 .../graph/drivers/ClusteringCoefficient.java    | 378 +++++--------------
 .../graph/drivers/ConnectedComponents.java      | 105 ++++++
 .../apache/flink/graph/drivers/EdgeList.java    |  92 +++++
 .../apache/flink/graph/drivers/Graph500.java    | 165 --------
 .../flink/graph/drivers/GraphMetrics.java       | 265 ++++---------
 .../org/apache/flink/graph/drivers/HITS.java    | 188 ++-------
 .../flink/graph/drivers/JaccardIndex.java       | 224 ++---------
 .../apache/flink/graph/drivers/PageRank.java    |  74 ++++
 .../flink/graph/drivers/SimpleDriver.java       |  65 ++++
 .../flink/graph/drivers/TriangleListing.java    | 362 +++++-------------
 .../drivers/parameter/IterationConvergence.java |  89 +++++
 .../graph/examples/ConnectedComponents.java     | 141 -------
 .../examples/GSASingleSourceShortestPaths.java  |   4 +-
 .../parameter/IterationConvergenceTest.java     |  66 ++++
 .../examples/ConnectedComponentsITCase.java     |  72 ----
 .../main/java/org/apache/flink/graph/Graph.java |  16 +-
 .../graph/library/ConnectedComponents.java      |   5 +-
 .../graph/library/GSAConnectedComponents.java   |   8 +-
 .../flink/graph/library/LabelPropagation.java   |   5 +-
 .../clustering/directed/TriangleListing.java    |   2 +-
 .../undirected/LocalClusteringCoefficient.java  |   2 +-
 .../graph/library/link_analysis/PageRank.java   |   8 +-
 .../graph/library/similarity/AdamicAdar.java    |   2 +-
 .../graph/library/similarity/JaccardIndex.java  |   2 +-
 .../apache/flink/graph/utils/GraphUtils.java    |  10 +-
 .../flink/graph/utils/NullValueEdgeMapper.java  |  32 --
 28 files changed, 919 insertions(+), 1536 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/Usage.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/Usage.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/Usage.java
index d923bf0..642fe5b 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/Usage.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/Usage.java
@@ -28,7 +28,6 @@ public class Usage {
 
 	private static final Class[] DRIVERS = new Class[]{
 		org.apache.flink.graph.drivers.ClusteringCoefficient.class,
-		org.apache.flink.graph.drivers.Graph500.class,
 		org.apache.flink.graph.drivers.GraphMetrics.class,
 		org.apache.flink.graph.drivers.HITS.class,
 		org.apache.flink.graph.drivers.JaccardIndex.class,
@@ -36,7 +35,6 @@ public class Usage {
 	};
 
 	private static final Class[] EXAMPLES = new Class[]{
-		org.apache.flink.graph.examples.ConnectedComponents.class,
 		org.apache.flink.graph.examples.EuclideanGraphWeighing.class,
 		org.apache.flink.graph.examples.GSASingleSourceShortestPaths.class,
 		org.apache.flink.graph.examples.IncrementalSSSP.class,

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/AdamicAdar.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/AdamicAdar.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/AdamicAdar.java
new file mode 100644
index 0000000..742c1de
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/AdamicAdar.java
@@ -0,0 +1,71 @@
+/*
+ * 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.flink.graph.drivers;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.commons.lang3.text.WordUtils;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.graph.library.similarity.AdamicAdar.Result;
+import org.apache.flink.types.CopyableValue;
+
+import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
+
+/**
+ * Driver for {@link org.apache.flink.graph.library.similarity.AdamicAdar}.
+ */
+public class AdamicAdar<K extends CopyableValue<K>, VV, EV>
+extends SimpleDriver<Result<K>>
+implements Driver<K, VV, EV>, CSV, Print {
+
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
+
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
+
+	@Override
+	public String getShortDescription() {
+		return "similarity score weighted by centerpoint degree";
+	}
+
+	@Override
+	public String getLongDescription() {
+		return WordUtils.wrap(new StrBuilder()
+			.appendln("Adamic-Adar measures the similarity between vertex neighborhoods and is " +
+				"computed as the sum of the inverse logarithm of centerpoint degree over shared " +
+				"neighbors.")
+			.appendNewLine()
+			.append("The algorithm result contains two vertex IDs and the similarity score.")
+			.toString(), 80);
+	}
+
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		int lp = littleParallelism.getValue().intValue();
+
+		result = graph
+			.run(new org.apache.flink.graph.library.similarity.AdamicAdar<K, VV, EV>()
+				.setLittleParallelism(lp));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ClusteringCoefficient.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ClusteringCoefficient.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ClusteringCoefficient.java
index 004390d..c463c0a 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ClusteringCoefficient.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ClusteringCoefficient.java
@@ -18,333 +18,127 @@
 
 package org.apache.flink.graph.drivers;
 
-import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAnalytic;
-import org.apache.flink.graph.GraphCsvReader;
-import org.apache.flink.graph.asm.translate.TranslateGraphIds;
-import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
-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.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
-
-import java.text.NumberFormat;
+import org.apache.flink.graph.asm.result.PrintableResult;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.ChoiceParameter;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
+import org.apache.flink.types.CopyableValue;
 
 import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
 
 /**
- * Driver for the library implementations of Global and Local Clustering Coefficient.
- *
- * This example reads a simple directed or undirected graph from a CSV file or
- * generates an RMat graph with the given scale and edge factor then calculates
- * the local clustering coefficient for each vertex and the global clustering
- * coefficient for the graph.
+ * Driver for directed and undirected clustering coefficient algorithm and analytics.
  *
+ * @see org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient
  * @see org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient
  * @see org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient
+ * @see org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient
  * @see org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient
  * @see org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient
  */
-public class ClusteringCoefficient {
-
-	private static final int DEFAULT_SCALE = 10;
-
-	private static final int DEFAULT_EDGE_FACTOR = 16;
-
-	private static final boolean DEFAULT_CLIP_AND_FLIP = true;
-
-	private static String getUsage(String message) {
-		return new StrBuilder()
-			.appendNewLine()
-			.appendln(WordUtils.wrap("The local clustering coefficient measures the connectedness of each" +
-				" vertex's neighborhood and the global clustering coefficient measures the connectedness of the graph." +
-				" Scores range from 0.0 (no edges between neighbors or vertices) to 1.0 (neighborhood or graph" +
-				" is a clique).", 80))
-			.appendNewLine()
-			.appendln(WordUtils.wrap("This algorithm returns tuples containing the vertex ID, the degree of" +
-				" the vertex, and the number of edges between vertex neighbors.", 80))
-			.appendNewLine()
-			.appendln("usage: ClusteringCoefficient --directed <true | false> --input <csv | rmat> --output <print | hash | csv>")
-			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --input csv --type <integer | string> [--simplify <true | false>] --input_filename FILENAME [--input_line_delimiter LINE_DELIMITER] [--input_field_delimiter FIELD_DELIMITER]")
-			.appendln("  --input rmat [--scale SCALE] [--edge_factor EDGE_FACTOR]")
-			.appendNewLine()
-			.appendln("  --output print")
-			.appendln("  --output hash")
-			.appendln("  --output csv --output_filename FILENAME [--output_line_delimiter LINE_DELIMITER] [--output_field_delimiter FIELD_DELIMITER]")
-			.appendNewLine()
-			.appendln("Usage error: " + message)
-			.toString();
-	}
-
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
-
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
-
-		if (! parameters.has("directed")) {
-			throw new ProgramParametrizationException(getUsage("must declare execution mode as '--directed true' or '--directed false'"));
-		}
-		boolean directedAlgorithm = parameters.getBoolean("directed");
-
-		int little_parallelism = parameters.getInt("little_parallelism", PARALLELISM_DEFAULT);
-
-		// global and local clustering coefficient results
-		GraphAnalytic gcc;
-		GraphAnalytic acc;
-		DataSet lcc;
-
-		switch (parameters.get("input", "")) {
-			case "csv": {
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				GraphCsvReader reader = Graph
-					.fromCsvReader(parameters.get("input_filename"), env)
-						.ignoreCommentsEdges("#")
-						.lineDelimiterEdges(lineDelimiter)
-						.fieldDelimiterEdges(fieldDelimiter);
-
-				switch (parameters.get("type", "")) {
-					case "integer": {
-						Graph<LongValue, NullValue, NullValue> graph = reader
-							.keyType(LongValue.class);
+public class ClusteringCoefficient<K extends Comparable<K> & CopyableValue<K>, VV, EV>
+extends SimpleDriver<PrintableResult>
+implements Driver<K, VV, EV>, CSV, Hash, Print {
 
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>()
-										.setParallelism(little_parallelism));
-							}
+	private static final String DIRECTED = "directed";
 
-							gcc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							acc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							lcc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(false)
-										.setParallelism(little_parallelism));
-							}
+	private static final String UNDIRECTED = "undirected";
 
-							gcc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							acc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							lcc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient<LongValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-					} break;
+	private ChoiceParameter order = new ChoiceParameter(this, "order")
+		.addChoices(DIRECTED, UNDIRECTED);
 
-					case "string": {
-						Graph<StringValue, NullValue, NullValue> graph = reader
-							.keyType(StringValue.class);
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
 
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<StringValue, NullValue, NullValue>()
-										.setParallelism(little_parallelism));
-							}
+	private GraphAnalytic<K, VV, EV, ? extends PrintableResult> globalClusteringCoefficient;
 
-							gcc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							acc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							lcc = graph
-								.run(new org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<StringValue, NullValue, NullValue>(false)
-										.setParallelism(little_parallelism));
-							}
+	private GraphAnalytic<K, VV, EV, ? extends PrintableResult> averageClusteringCoefficient;
 
-							gcc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							acc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-							lcc = graph
-								.run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient<StringValue, NullValue, NullValue>()
-									.setLittleParallelism(little_parallelism));
-						}
-					} break;
-
-					default:
-						throw new ProgramParametrizationException(getUsage("invalid CSV type"));
-				}
-			} break;
-
-			case "rmat": {
-				int scale = parameters.getInt("scale", DEFAULT_SCALE);
-				int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-				RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-				long vertexCount = 1L << scale;
-				long edgeCount = vertexCount * edgeFactor;
-
-				Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-					.setParallelism(little_parallelism)
-					.generate();
-
-				if (directedAlgorithm) {
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> newGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>()
-								.setParallelism(little_parallelism));
-
-						gcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						acc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						lcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setIncludeZeroDegreeVertices(false)
-								.setLittleParallelism(little_parallelism));
-					} else {
-						Graph<IntValue, NullValue, NullValue> newGraph = graph
-							.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue())
-								.setParallelism(little_parallelism))
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<IntValue, NullValue, NullValue>()
-								.setParallelism(little_parallelism));
-
-						gcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						acc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						lcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setIncludeZeroDegreeVertices(false)
-								.setLittleParallelism(little_parallelism));
-					}
-				} else {
-					boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP);
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
 
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> newGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(clipAndFlip)
-								.setParallelism(little_parallelism));
+	@Override
+	public String getShortDescription() {
+		return "measure the connectedness of vertex neighborhoods";
+	}
 
-						gcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						acc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						lcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient<LongValue, NullValue, NullValue>()
-								.setIncludeZeroDegreeVertices(false)
-								.setLittleParallelism(little_parallelism));
-					} else {
-						Graph<IntValue, NullValue, NullValue> newGraph = graph
-							.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue())
-								.setParallelism(little_parallelism))
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<IntValue, NullValue, NullValue>(clipAndFlip)
-								.setParallelism(little_parallelism));
+	@Override
+	public String getLongDescription() {
+		return WordUtils.wrap(new StrBuilder()
+			.appendln("The local clustering coefficient measures the connectedness of each " +
+				"vertex's neighborhood. The global clustering coefficient measures the " +
+				"connected of the graph. The average clustering coefficient is the mean local " +
+				"clustering coefficient. Each score ranges from 0.0 (no edges between vertex " +
+				"neighbors) to 1.0 (neighborhood or graph is a clique).")
+			.appendNewLine()
+			.append("The algorithm result contains the vertex ID, degree, and number of edges " +
+				"connecting neighbors.")
+			.toString(), 80);
+	}
 
-						gcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						acc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-						lcc = newGraph
-							.run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient<IntValue, NullValue, NullValue>()
-								.setIncludeZeroDegreeVertices(false)
-								.setLittleParallelism(little_parallelism));
-					}
-				}
-			} break;
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		int lp = littleParallelism.getValue().intValue();
 
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid input type"));
-		}
+		switch (order.getValue()) {
+			case DIRECTED:
+				result = graph
+					.run(new org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 
-		switch (parameters.get("output", "")) {
-			case "print":
-				if (directedAlgorithm) {
-					for (Object e: lcc.collect()) {
-						org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient.Result result =
-							(org.apache.flink.graph.library.clustering.directed.LocalClusteringCoefficient.Result)e;
-						System.out.println(result.toPrintableString());
-					}
-				} else {
-					for (Object e: lcc.collect()) {
-						org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result result =
-							(org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result)e;
-						System.out.println(result.toPrintableString());
-					}
-				}
-				break;
+				globalClusteringCoefficient = graph
+					.run(new org.apache.flink.graph.library.clustering.directed.GlobalClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 
-			case "hash":
-				System.out.println(DataSetUtils.checksumHashCode(lcc));
+				averageClusteringCoefficient = graph
+					.run(new org.apache.flink.graph.library.clustering.directed.AverageClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 				break;
 
-			case "csv":
-				String filename = parameters.get("output_filename");
+			case UNDIRECTED:
+				result = graph
+					.run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
+				globalClusteringCoefficient = graph
+					.run(new org.apache.flink.graph.library.clustering.undirected.GlobalClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				lcc.writeAsCsv(filename, lineDelimiter, fieldDelimiter);
-
-				env.execute("Clustering Coefficient");
+				averageClusteringCoefficient = graph
+					.run(new org.apache.flink.graph.library.clustering.undirected.AverageClusteringCoefficient<K, VV, EV>()
+						.setLittleParallelism(lp));
 				break;
-
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid output type"));
 		}
+	}
 
-		System.out.println(gcc.getResult());
-		System.out.println(acc.getResult());
+	@Override
+	public void hash(String executionName) throws Exception {
+		super.hash(executionName);
+		printAnalytics();
+	}
 
-		JobExecutionResult result = env.getLastJobExecutionResult();
+	@Override
+	public void print(String executionName) throws Exception {
+		super.print(executionName);
+		printAnalytics();
+	}
+
+	@Override
+	public void writeCSV(String filename, String lineDelimiter, String fieldDelimiter) {
+		super.writeCSV(filename, lineDelimiter, fieldDelimiter);
+		printAnalytics();
+	}
 
-		NumberFormat nf = NumberFormat.getInstance();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
+	private void printAnalytics() {
+		System.out.println(globalClusteringCoefficient.getResult().toPrintableString());
+		System.out.println(averageClusteringCoefficient.getResult().toPrintableString());
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
new file mode 100644
index 0000000..32263cf
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/ConnectedComponents.java
@@ -0,0 +1,105 @@
+/*
+ * 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.flink.graph.drivers;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum;
+import org.apache.flink.graph.asm.dataset.Collect;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.graph.library.GSAConnectedComponents;
+
+import java.util.List;
+
+/**
+ * Driver for {@link org.apache.flink.graph.library.GSAConnectedComponents}.
+ *
+ * The gather-sum-apply implementation is used because scatter-gather does not
+ * handle object reuse (see FLINK-5891).
+ */
+public class ConnectedComponents<K extends Comparable<K>, VV, EV>
+extends ParameterizedBase
+implements Driver<K, VV, EV>, CSV, Hash, Print {
+
+	private DataSet<Vertex<K, K>> components;
+
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
+
+	@Override
+	public String getShortDescription() {
+		return "ConnectedComponents";
+	}
+
+	@Override
+	public String getLongDescription() {
+		return "ConnectedComponents";
+	}
+
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		components = graph
+			.mapVertices(new MapVertices<K, VV>())
+			.run(new GSAConnectedComponents<K, K, EV>(Integer.MAX_VALUE));
+	}
+
+	@Override
+	public void hash(String executionName) throws Exception {
+		Checksum checksum = new ChecksumHashCode<Vertex<K, K>>()
+			.run(components)
+			.execute(executionName);
+
+		System.out.println(checksum);
+	}
+
+	@Override
+	public void print(String executionName) throws Exception {
+		Collect<Vertex<K, K>> collector = new Collect<>();
+
+		// Refactored due to openjdk7 compile error: https://travis-ci.org/greghogan/flink/builds/200487761
+		List<Vertex<K, K>> records = collector.run(components).execute(executionName);
+
+		for (Vertex<K, K> result : records) {
+			System.out.println(result);
+		}
+	}
+
+	@Override
+	public void writeCSV(String filename, String lineDelimiter, String fieldDelimiter) {
+		components
+			.writeAsCsv(filename, lineDelimiter, fieldDelimiter)
+				.name("CSV: " + filename);
+	}
+
+	private static final class MapVertices<T, VT>
+	implements MapFunction<Vertex<T, VT>, T> {
+		@Override
+		public T map(Vertex<T, VT> value) throws Exception {
+			return value.f0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
new file mode 100644
index 0000000..85f32c3
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/EdgeList.java
@@ -0,0 +1,92 @@
+/*
+ * 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.flink.graph.drivers;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum;
+import org.apache.flink.graph.asm.dataset.Collect;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+
+import java.util.List;
+
+/**
+ * Convert a {@link Graph} to the {@link DataSet} of {@link Edge}s.
+ */
+public class EdgeList<K, VV, EV>
+extends ParameterizedBase
+implements Driver<K, VV, EV>, CSV, Hash, Print {
+
+	private DataSet<Edge<K, EV>> edges;
+
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
+
+	@Override
+	public String getShortDescription() {
+		return "the edge list";
+	}
+
+	@Override
+	public String getLongDescription() {
+		return "Pass-through of the graph's edge list.";
+	}
+
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		edges = graph
+			.getEdges();
+	}
+
+	@Override
+	public void hash(String executionName) throws Exception {
+		Checksum checksum = new ChecksumHashCode<Edge<K, EV>>()
+			.run(edges)
+			.execute(executionName);
+
+		System.out.println(checksum);
+	}
+
+	@Override
+	public void print(String executionName) throws Exception {
+		Collect<Edge<K, EV>> collector = new Collect<>();
+
+		// Refactored due to openjdk7 compile error: https://travis-ci.org/greghogan/flink/builds/200487761
+		List<Edge<K, EV>> records = collector.run(edges).execute(executionName);
+
+		for (Edge<K, EV> result : records) {
+			System.out.println(result);
+		}
+
+	}
+
+	@Override
+	public void writeCSV(String filename, String lineDelimiter, String fieldDelimiter) {
+		edges
+			.writeAsCsv(filename, lineDelimiter, fieldDelimiter)
+				.name("CSV: " + filename);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/Graph500.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/Graph500.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/Graph500.java
deleted file mode 100644
index c2abbf7..0000000
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/Graph500.java
+++ /dev/null
@@ -1,165 +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.flink.graph.drivers;
-
-import org.apache.commons.lang3.StringEscapeUtils;
-import org.apache.commons.lang3.text.StrBuilder;
-import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
-import org.apache.flink.graph.Graph;
-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.types.LongValue;
-import org.apache.flink.types.NullValue;
-
-import java.text.NumberFormat;
-
-/**
- * Generate an RMat graph for Graph 500.
- *
- * Note that this does not yet implement permutation of vertex labels or edges.
- *
- * @see <a href="http://www.graph500.org/specifications">Graph 500</a>
- */
-public class Graph500 {
-
-	private static final int DEFAULT_SCALE = 10;
-
-	private static final int DEFAULT_EDGE_FACTOR = 16;
-
-	private static final boolean DEFAULT_CLIP_AND_FLIP = true;
-
-	private static String getUsage(String message) {
-		return new StrBuilder()
-			.appendNewLine()
-			.appendln("A Graph500 generator using the Recursive Matrix (RMat) graph generator.")
-			.appendNewLine()
-			.appendln(WordUtils.wrap("The graph matrix contains 2^scale vertices although not every vertex will" +
-				" be represented in an edge. The number of edges is edge_factor * 2^scale edges" +
-				" although some edges may be duplicates.", 80))
-			.appendNewLine()
-			.appendln("Note: this does not yet implement permutation of vertex labels or edges.")
-			.appendNewLine()
-			.appendln("usage: Graph500 --directed <true | false> --simplify <true | false> --output <print | hash | csv [options]>")
-			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --output print")
-			.appendln("  --output hash")
-			.appendln("  --output csv --output_filename FILENAME [--output_line_delimiter LINE_DELIMITER] [--output_field_delimiter FIELD_DELIMITER]")
-			.appendNewLine()
-			.appendln("Usage error: " + message)
-			.toString();
-	}
-
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
-
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
-
-		if (! parameters.has("directed")) {
-			throw new ProgramParametrizationException(getUsage("must declare execution mode as '--directed true' or '--directed false'"));
-		}
-		boolean directed = parameters.getBoolean("directed");
-
-		if (! parameters.has("simplify")) {
-			throw new ProgramParametrizationException(getUsage("must declare '--simplify true' or '--simplify false'"));
-		}
-		boolean simplify = parameters.getBoolean("simplify");
-
-
-		// Generate RMat graph
-		int scale = parameters.getInt("scale", DEFAULT_SCALE);
-		int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-		RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-		long vertexCount = 1L << scale;
-		long edgeCount = vertexCount * edgeFactor;
-
-		boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP);
-
-		Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-			.generate();
-
-		if (directed) {
-			if (simplify) {
-				graph = graph
-					.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>());
-			}
-		} else {
-			if (simplify) {
-				graph = graph
-					.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(clipAndFlip));
-			} else {
-				graph = graph.getUndirected();
-			}
-		}
-
-		DataSet<Tuple2<LongValue, LongValue>> edges = graph
-			.getEdges()
-			.project(0, 1);
-
-		// Print, hash, or write RMat graph to disk
-		switch (parameters.get("output", "")) {
-		case "print":
-			System.out.println();
-			edges.print();
-			break;
-
-		case "hash":
-			System.out.println();
-			System.out.println(DataSetUtils.checksumHashCode(edges));
-			break;
-
-		case "csv":
-			String filename = parameters.getRequired("output_filename");
-
-			String lineDelimiter = StringEscapeUtils.unescapeJava(
-				parameters.get("output_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-			String fieldDelimiter = StringEscapeUtils.unescapeJava(
-				parameters.get("output_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-			edges.writeAsCsv(filename, lineDelimiter, fieldDelimiter);
-
-			env.execute("Graph500");
-			break;
-		default:
-			throw new ProgramParametrizationException(getUsage("invalid output type"));
-		}
-
-		JobExecutionResult result = env.getLastJobExecutionResult();
-
-		NumberFormat nf = NumberFormat.getInstance();
-		System.out.println();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/GraphMetrics.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/GraphMetrics.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/GraphMetrics.java
index 9b246df..cc5a894 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/GraphMetrics.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/GraphMetrics.java
@@ -18,224 +18,109 @@
 
 package org.apache.flink.graph.drivers;
 
-import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.text.StrBuilder;
-import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.GraphAnalytic;
-import org.apache.flink.graph.GraphCsvReader;
-import org.apache.flink.graph.asm.translate.TranslateGraphIds;
-import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
-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.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
-
-import java.text.NumberFormat;
+import org.apache.flink.graph.asm.result.PrintableResult;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.ChoiceParameter;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+import org.apache.flink.types.CopyableValue;
 
 /**
- * Computes vertex and edge metrics on a directed or undirected graph.
+ * Driver for directed and undirected graph metrics analytics.
  *
  * @see org.apache.flink.graph.library.metric.directed.EdgeMetrics
  * @see org.apache.flink.graph.library.metric.directed.VertexMetrics
  * @see org.apache.flink.graph.library.metric.undirected.EdgeMetrics
  * @see org.apache.flink.graph.library.metric.undirected.VertexMetrics
  */
-public class GraphMetrics {
+public class GraphMetrics<K extends Comparable<K> & CopyableValue<K>, VV, EV>
+extends ParameterizedBase
+implements Driver<K, VV, EV>, Hash, Print {
 
-	private static final int DEFAULT_SCALE = 10;
+	private static final String DIRECTED = "directed";
 
-	private static final int DEFAULT_EDGE_FACTOR = 16;
+	private static final String UNDIRECTED = "undirected";
 
-	private static final boolean DEFAULT_CLIP_AND_FLIP = true;
+	private ChoiceParameter order = new ChoiceParameter(this, "order")
+		.addChoices(DIRECTED, UNDIRECTED);
 
-	private static String getUsage(String message) {
+	private GraphAnalytic<K, VV, EV, ? extends PrintableResult> vertexMetrics;
+
+	private GraphAnalytic<K, VV, EV, ? extends PrintableResult> edgeMetrics;
+
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
+
+	@Override
+	public String getShortDescription() {
+		return "compute vertex and edge metrics";
+	}
+
+	@Override
+	public String getLongDescription() {
 		return new StrBuilder()
+			.appendln("Computes metrics on a directed or undirected graph.")
 			.appendNewLine()
-			.appendln(WordUtils.wrap("Computes vertex and edge metrics on a directed or undirected graph.", 80))
-			.appendNewLine()
-			.appendln("usage: GraphMetrics --directed <true | false> --input <csv | rmat>")
+			.appendln("Vertex metrics:")
+			.appendln("- number of vertices")
+			.appendln("- number of edges")
+			.appendln("- number of unidirectional edges (directed only)")
+			.appendln("- number of bidirectional edges (directed only)")
+			.appendln("- average degree")
+			.appendln("- number of triplets")
+			.appendln("- maximum degree")
+			.appendln("- maximum out degree (directed only)")
+			.appendln("- maximum in degree (directed only)")
+			.appendln("- maximum number of triplets")
 			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --input csv --type <integer | string> [--simplify <true | false>] --input_filename FILENAME [--input_line_delimiter LINE_DELIMITER] [--input_field_delimiter FIELD_DELIMITER]")
-			.appendln("  --input rmat [--scale SCALE] [--edge_factor EDGE_FACTOR]")
-			.appendNewLine()
-			.appendln("Usage error: " + message)
+			.appendln("Edge metrics:")
+			.appendln("- number of triangle triplets")
+			.appendln("- number of rectangle triplets")
+			.appendln("- maximum number of triangle triplets")
+			.append("- maximum number of rectangle triplets")
 			.toString();
 	}
 
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		switch (order.getValue()) {
+			case DIRECTED:
+				vertexMetrics = graph
+					.run(new org.apache.flink.graph.library.metric.directed.VertexMetrics<K, VV, EV>());
 
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
+				edgeMetrics = graph
+					.run(new org.apache.flink.graph.library.metric.directed.EdgeMetrics<K, VV, EV>());
+				break;
 
-		if (! parameters.has("directed")) {
-			throw new ProgramParametrizationException(getUsage("must declare execution mode as '--directed true' or '--directed false'"));
-		}
-		boolean directedAlgorithm = parameters.getBoolean("directed");
-
-		GraphAnalytic vm;
-		GraphAnalytic em;
-
-		switch (parameters.get("input", "")) {
-			case "csv": {
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				GraphCsvReader reader = Graph
-					.fromCsvReader(parameters.getRequired("input_filename"), env)
-						.ignoreCommentsEdges("#")
-						.lineDelimiterEdges(lineDelimiter)
-						.fieldDelimiterEdges(fieldDelimiter);
-
-				switch (parameters.get("type", "")) {
-					case "integer": {
-						Graph<LongValue, NullValue, NullValue> graph = reader
-							.keyType(LongValue.class);
-
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>());
-							}
-
-							vm = graph
-								.run(new org.apache.flink.graph.library.metric.directed.VertexMetrics<LongValue, NullValue, NullValue>());
-							em = graph
-								.run(new org.apache.flink.graph.library.metric.directed.EdgeMetrics<LongValue, NullValue, NullValue>());
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(false));
-							}
-
-							vm = graph
-								.run(new org.apache.flink.graph.library.metric.undirected.VertexMetrics<LongValue, NullValue, NullValue>());
-							em = graph
-								.run(new org.apache.flink.graph.library.metric.undirected.EdgeMetrics<LongValue, NullValue, NullValue>());
-						}
-					} break;
-
-					case "string": {
-						Graph<StringValue, NullValue, NullValue> graph = reader
-							.keyType(StringValue.class);
-
-						if (directedAlgorithm) {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.directed.Simplify<StringValue, NullValue, NullValue>());
-							}
-
-							vm = graph
-								.run(new org.apache.flink.graph.library.metric.directed.VertexMetrics<StringValue, NullValue, NullValue>());
-							em = graph
-								.run(new org.apache.flink.graph.library.metric.directed.EdgeMetrics<StringValue, NullValue, NullValue>());
-						} else {
-							if (parameters.getBoolean("simplify", false)) {
-								graph = graph
-									.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<StringValue, NullValue, NullValue>(false));
-							}
-
-							vm = graph
-								.run(new org.apache.flink.graph.library.metric.undirected.VertexMetrics<StringValue, NullValue, NullValue>());
-							em = graph
-								.run(new org.apache.flink.graph.library.metric.undirected.EdgeMetrics<StringValue, NullValue, NullValue>());
-						}
-					} break;
-
-					default:
-						throw new ProgramParametrizationException(getUsage("invalid CSV type"));
-				}
-				} break;
-
-			case "rmat": {
-				int scale = parameters.getInt("scale", DEFAULT_SCALE);
-				int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-				RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-				long vertexCount = 1L << scale;
-				long edgeCount = vertexCount * edgeFactor;
-
-
-				Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-					.generate();
-
-				if (directedAlgorithm) {
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> newGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<LongValue, NullValue, NullValue>());
-
-						vm = newGraph
-							.run(new org.apache.flink.graph.library.metric.directed.VertexMetrics<LongValue, NullValue, NullValue>());
-						em = newGraph
-							.run(new org.apache.flink.graph.library.metric.directed.EdgeMetrics<LongValue, NullValue, NullValue>());
-					} else {
-						Graph<IntValue, NullValue, NullValue> newGraph = graph
-							.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue()))
-							.run(new org.apache.flink.graph.asm.simple.directed.Simplify<IntValue, NullValue, NullValue>());
-
-						vm = newGraph
-							.run(new org.apache.flink.graph.library.metric.directed.VertexMetrics<IntValue, NullValue, NullValue>());
-						em = newGraph
-							.run(new org.apache.flink.graph.library.metric.directed.EdgeMetrics<IntValue, NullValue, NullValue>());
-					}
-				} else {
-					boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP);
-
-					if (scale > 32) {
-						Graph<LongValue, NullValue, NullValue> newGraph = graph
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(clipAndFlip));
-
-						vm = newGraph
-							.run(new org.apache.flink.graph.library.metric.undirected.VertexMetrics<LongValue, NullValue, NullValue>());
-						em = newGraph
-							.run(new org.apache.flink.graph.library.metric.undirected.EdgeMetrics<LongValue, NullValue, NullValue>());
-					} else {
-						Graph<IntValue, NullValue, NullValue> newGraph = graph
-							.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue()))
-							.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<IntValue, NullValue, NullValue>(clipAndFlip));
-
-						vm = newGraph
-							.run(new org.apache.flink.graph.library.metric.undirected.VertexMetrics<IntValue, NullValue, NullValue>());
-						em = newGraph
-							.run(new org.apache.flink.graph.library.metric.undirected.EdgeMetrics<IntValue, NullValue, NullValue>());
-					}
-				}
-				} break;
-
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid input type"));
+			case UNDIRECTED:
+				vertexMetrics = graph
+					.run(new org.apache.flink.graph.library.metric.undirected.VertexMetrics<K, VV, EV>());
+
+				edgeMetrics = graph
+					.run(new org.apache.flink.graph.library.metric.undirected.EdgeMetrics<K, VV, EV>());
+				break;
 		}
+	}
 
-		env.execute("Graph Metrics");
+	@Override
+	public void hash(String executionName) throws Exception {
+		print(executionName);
+	}
 
-		System.out.println();
-		System.out.print("Vertex metrics:\n  ");
-		System.out.println(vm.getResult().toString().replace(";", "\n "));
-		System.out.println();
-		System.out.print("Edge metrics:\n  ");
-		System.out.println(em.getResult().toString().replace(";", "\n "));
+	@Override
+	public void print(String executionName) throws Exception {
+		vertexMetrics.execute(executionName);
 
-		JobExecutionResult result = env.getLastJobExecutionResult();
+		System.out.print("Vertex metrics:\n  ");
+		System.out.println(vertexMetrics.getResult().toPrintableString().replace(";", "\n "));
 
-		NumberFormat nf = NumberFormat.getInstance();
 		System.out.println();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
+		System.out.print("Edge metrics:\n  ");
+		System.out.println(edgeMetrics.getResult().toPrintableString().replace(";", "\n "));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/HITS.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/HITS.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/HITS.java
index db27f0e..6081fea 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/HITS.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/HITS.java
@@ -18,177 +18,51 @@
 
 package org.apache.flink.graph.drivers;
 
-import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphCsvReader;
-import org.apache.flink.graph.asm.simple.directed.Simplify;
-import org.apache.flink.graph.asm.translate.TranslateGraphIds;
-import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
-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.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.IterationConvergence;
 import org.apache.flink.graph.library.link_analysis.HITS.Result;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
-
-import java.text.NumberFormat;
 
 /**
- * Driver for the library implementation of HITS (Hubs and Authorities).
- *
- * This example reads a simple, undirected graph from a CSV file or generates
- * an undirected RMat graph with the given scale and edge factor then calculates
- * hub and authority scores for each vertex.
- *
- * @see org.apache.flink.graph.library.link_analysis.HITS
+ * Driver for {@link org.apache.flink.graph.library.link_analysis.HITS}.
  */
-public class HITS {
+public class HITS<K, VV, EV>
+extends SimpleDriver<Result<K>>
+implements Driver<K, VV, EV>, CSV, Print {
 
 	private static final int DEFAULT_ITERATIONS = 10;
 
-	private static final int DEFAULT_SCALE = 10;
+	private IterationConvergence iterationConvergence = new IterationConvergence(this, DEFAULT_ITERATIONS);
 
-	private static final int DEFAULT_EDGE_FACTOR = 16;
-
-	private static String getUsage(String message) {
-		return new StrBuilder()
-			.appendNewLine()
-			.appendln(WordUtils.wrap("Hyperlink-Induced Topic Search computes two interdependent" +
-				" scores for every vertex in a directed graph. A good \"hub\" links to good \"authorities\"" +
-				" and good \"authorities\" are linked from good \"hubs\".", 80))
-			.appendNewLine()
-			.appendln("usage: HITS --input <csv | rmat> --output <print | hash | csv>")
-			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --input csv --type <integer | string> --input_filename FILENAME [--input_line_delimiter LINE_DELIMITER] [--input_field_delimiter FIELD_DELIMITER]")
-			.appendln("  --input rmat [--scale SCALE] [--edge_factor EDGE_FACTOR]")
-			.appendNewLine()
-			.appendln("  --output print")
-			.appendln("  --output hash")
-			.appendln("  --output csv --output_filename FILENAME [--output_line_delimiter LINE_DELIMITER] [--output_field_delimiter FIELD_DELIMITER]")
-			.appendNewLine()
-			.appendln("Usage error: " + message)
-			.toString();
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
 	}
 
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
-
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
-
-		int iterations = parameters.getInt("iterations", DEFAULT_ITERATIONS);
-
-		DataSet hits;
-
-		switch (parameters.get("input", "")) {
-			case "csv": {
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				GraphCsvReader reader = Graph
-					.fromCsvReader(parameters.getRequired("input_filename"), env)
-						.ignoreCommentsEdges("#")
-						.lineDelimiterEdges(lineDelimiter)
-						.fieldDelimiterEdges(fieldDelimiter);
-
-				switch (parameters.get("type", "")) {
-					case "integer": {
-						hits = reader
-							.keyType(LongValue.class)
-							.run(new org.apache.flink.graph.library.link_analysis.HITS<LongValue, NullValue, NullValue>(iterations));
-					} break;
-
-					case "string": {
-						hits = reader
-							.keyType(StringValue.class)
-							.run(new org.apache.flink.graph.library.link_analysis.HITS<StringValue, NullValue, NullValue>(iterations));
-					} break;
-
-					default:
-						throw new ProgramParametrizationException(getUsage("invalid CSV type"));
-				}
-				} break;
-
-			case "rmat": {
-				int scale = parameters.getInt("scale", DEFAULT_SCALE);
-				int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-				RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-				long vertexCount = 1L << scale;
-				long edgeCount = vertexCount * edgeFactor;
-
-				Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-					.generate();
-
-				if (scale > 32) {
-					hits = graph
-						.run(new Simplify<LongValue, NullValue, NullValue>())
-						.run(new org.apache.flink.graph.library.link_analysis.HITS<LongValue, NullValue, NullValue>(iterations));
-				} else {
-					hits = graph
-						.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue()))
-						.run(new Simplify<IntValue, NullValue, NullValue>())
-						.run(new org.apache.flink.graph.library.link_analysis.HITS<IntValue, NullValue, NullValue>(iterations));
-				}
-				} break;
-
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid input type"));
-		}
-
-		switch (parameters.get("output", "")) {
-			case "print":
-				System.out.println();
-				for (Object e: hits.collect()) {
-					System.out.println(((Result)e).toPrintableString());
-				}
-				break;
-
-			case "hash":
-				System.out.println();
-				System.out.println(DataSetUtils.checksumHashCode(hits));
-				break;
-
-			case "csv":
-				String filename = parameters.getRequired("output_filename");
-
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				hits.writeAsCsv(filename, lineDelimiter, fieldDelimiter);
-
-				env.execute("HITS");
-				break;
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid output type"));
-		}
+	@Override
+	public String getShortDescription() {
+		return "score vertices as hubs and authorities";
+	}
 
-		JobExecutionResult result = env.getLastJobExecutionResult();
+	@Override
+	public String getLongDescription() {
+		return WordUtils.wrap(new StrBuilder()
+			.appendln("Hyperlink-Induced Topic Search computes two interdependent scores for " +
+				"each vertex in a directed graph. A good \"hub\" links to good \"authorities\" " +
+				"and good \"authorities\" are linked to from good \"hubs\".")
+			.appendNewLine()
+			.append("The result contains the vertex ID, hub score, and authority score.")
+			.toString(), 80);
+	}
 
-		NumberFormat nf = NumberFormat.getInstance();
-		System.out.println();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		result = graph
+			.run(new org.apache.flink.graph.library.link_analysis.HITS<K, VV, EV>(
+				iterationConvergence.getValue().iterations,
+				iterationConvergence.getValue().convergenceThreshold));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/JaccardIndex.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/JaccardIndex.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/JaccardIndex.java
index 09479a6..1c836ea 100644
--- a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/JaccardIndex.java
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/JaccardIndex.java
@@ -18,211 +18,57 @@
 
 package org.apache.flink.graph.drivers;
 
-import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.commons.lang3.text.WordUtils;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.CsvOutputFormat;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.client.program.ProgramParametrizationException;
 import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphCsvReader;
-import org.apache.flink.graph.asm.simple.undirected.Simplify;
-import org.apache.flink.graph.asm.translate.TranslateGraphIds;
-import org.apache.flink.graph.asm.translate.translators.LongValueToUnsignedIntValue;
-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.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Hash;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.LongParameter;
 import org.apache.flink.graph.library.similarity.JaccardIndex.Result;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
-
-import java.text.NumberFormat;
+import org.apache.flink.types.CopyableValue;
 
 import static org.apache.flink.api.common.ExecutionConfig.PARALLELISM_DEFAULT;
 
 /**
- * Driver for the library implementation of Jaccard Index.
- *
- * This example reads a simple, undirected graph from a CSV file or generates
- * an undirected RMat graph with the given scale and edge factor then calculates
- * all non-zero Jaccard Index similarity scores between vertices.
- *
- * @see org.apache.flink.graph.library.similarity.JaccardIndex
+ * Driver for {@link org.apache.flink.graph.library.similarity.JaccardIndex}.
  */
-public class JaccardIndex {
+public class JaccardIndex<K extends CopyableValue<K>, VV, EV>
+extends SimpleDriver<Result<K>>
+implements Driver<K, VV, EV>, CSV, Hash, Print {
 
-	private static final int DEFAULT_SCALE = 10;
+	private LongParameter littleParallelism = new LongParameter(this, "little_parallelism")
+		.setDefaultValue(PARALLELISM_DEFAULT);
 
-	private static final int DEFAULT_EDGE_FACTOR = 16;
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
 
-	private static final boolean DEFAULT_CLIP_AND_FLIP = true;
+	@Override
+	public String getShortDescription() {
+		return "similarity score as fraction of common neighbors";
+	}
 
-	private static String getUsage(String message) {
-		return new StrBuilder()
-			.appendNewLine()
-			.appendln(WordUtils.wrap("The Jaccard Index measures the similarity between vertex" +
-				" neighborhoods and is computed as the number of shared neighbors divided by the number of" +
-				" distinct neighbors. Scores range from 0.0 (no shared neighbors) to 1.0 (all neighbors are" +
-				" shared).", 80))
-			.appendNewLine()
-			.appendln(WordUtils.wrap("This algorithm returns 4-tuples containing two vertex IDs, the" +
-				" number of shared neighbors, and the number of distinct neighbors.", 80))
-			.appendNewLine()
-			.appendln("usage: JaccardIndex --input <csv | rmat> --output <print | hash | csv>")
-			.appendNewLine()
-			.appendln("options:")
-			.appendln("  --input csv --type <integer | string> [--simplify <true | false>] --input_filename FILENAME [--input_line_delimiter LINE_DELIMITER] [--input_field_delimiter FIELD_DELIMITER]")
-			.appendln("  --input rmat [--scale SCALE] [--edge_factor EDGE_FACTOR]")
-			.appendNewLine()
-			.appendln("  --output print")
-			.appendln("  --output hash")
-			.appendln("  --output csv --output_filename FILENAME [--output_line_delimiter LINE_DELIMITER] [--output_field_delimiter FIELD_DELIMITER]")
+	@Override
+	public String getLongDescription() {
+		return WordUtils.wrap(new StrBuilder()
+			.appendln("Jaccard Index measures the similarity between vertex neighborhoods and " +
+				"is computed as the number of shared neighbors divided by the number of " +
+				"distinct neighbors. Scores range from 0.0 (no shared neighbors) to 1.0 (all " +
+				"neighbors are shared).")
 			.appendNewLine()
-			.appendln("Usage error: " + message)
-			.toString();
+			.append("The result contains two vertex IDs, the number of shared neighbors, and " +
+				"the number of distinct neighbors.")
+			.toString(), 80);
 	}
 
-	public static void main(String[] args) throws Exception {
-		// Set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableObjectReuse();
-
-		ParameterTool parameters = ParameterTool.fromArgs(args);
-		env.getConfig().setGlobalJobParameters(parameters);
-
-		int little_parallelism = parameters.getInt("little_parallelism", PARALLELISM_DEFAULT);
-
-		DataSet ji;
-
-		switch (parameters.get("input", "")) {
-			case "csv": {
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("input_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				GraphCsvReader reader = Graph
-					.fromCsvReader(parameters.getRequired("input_filename"), env)
-						.ignoreCommentsEdges("#")
-						.lineDelimiterEdges(lineDelimiter)
-						.fieldDelimiterEdges(fieldDelimiter);
-
-				switch (parameters.get("type", "")) {
-					case "integer": {
-						Graph<LongValue, NullValue, NullValue> graph = reader
-							.keyType(LongValue.class);
-
-						if (parameters.getBoolean("simplify", false)) {
-							graph = graph
-								.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<LongValue, NullValue, NullValue>(false)
-									.setParallelism(little_parallelism));
-						}
-
-						ji = graph
-							.run(new org.apache.flink.graph.library.similarity.JaccardIndex<LongValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					} break;
-
-					case "string": {
-						Graph<StringValue, NullValue, NullValue> graph = reader
-							.keyType(StringValue.class);
-
-						if (parameters.getBoolean("simplify", false)) {
-							graph = graph
-								.run(new org.apache.flink.graph.asm.simple.undirected.Simplify<StringValue, NullValue, NullValue>(false)
-									.setParallelism(little_parallelism));
-						}
-
-						ji = graph
-							.run(new org.apache.flink.graph.library.similarity.JaccardIndex<StringValue, NullValue, NullValue>()
-								.setLittleParallelism(little_parallelism));
-					} break;
-
-					default:
-						throw new ProgramParametrizationException(getUsage("invalid CSV type"));
-				}
-				} break;
-
-			case "rmat": {
-				int scale = parameters.getInt("scale", DEFAULT_SCALE);
-				int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR);
-
-				RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
-
-				long vertexCount = 1L << scale;
-				long edgeCount = vertexCount * edgeFactor;
-
-				Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
-					.setParallelism(little_parallelism)
-					.generate();
-
-				boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP);
-
-				if (scale > 32) {
-					ji = graph
-						.run(new Simplify<LongValue, NullValue, NullValue>(clipAndFlip)
-							.setParallelism(little_parallelism))
-						.run(new org.apache.flink.graph.library.similarity.JaccardIndex<LongValue, NullValue, NullValue>()
-							.setLittleParallelism(little_parallelism));
-				} else {
-					ji = graph
-						.run(new TranslateGraphIds<LongValue, IntValue, NullValue, NullValue>(new LongValueToUnsignedIntValue())
-							.setParallelism(little_parallelism))
-						.run(new Simplify<IntValue, NullValue, NullValue>(clipAndFlip)
-							.setParallelism(little_parallelism))
-						.run(new org.apache.flink.graph.library.similarity.JaccardIndex<IntValue, NullValue, NullValue>()
-							.setLittleParallelism(little_parallelism));
-				}
-				} break;
-
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid input type"));
-		}
-
-		switch (parameters.get("output", "")) {
-			case "print":
-				System.out.println();
-				for (Object e: ji.collect()) {
-					Result result = (Result)e;
-					System.out.println(result.toPrintableString());
-				}
-				break;
-
-			case "hash":
-				System.out.println();
-				System.out.println(DataSetUtils.checksumHashCode(ji));
-				break;
-
-			case "csv":
-				String filename = parameters.getRequired("output_filename");
-
-				String lineDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_line_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER));
-
-				String fieldDelimiter = StringEscapeUtils.unescapeJava(
-					parameters.get("output_field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER));
-
-				ji.writeAsCsv(filename, lineDelimiter, fieldDelimiter);
-
-				env.execute("Jaccard Index");
-				break;
-
-			default:
-				throw new ProgramParametrizationException(getUsage("invalid output type"));
-		}
-
-		JobExecutionResult result = env.getLastJobExecutionResult();
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		int lp = littleParallelism.getValue().intValue();
 
-		NumberFormat nf = NumberFormat.getInstance();
-		System.out.println();
-		System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms");
+		result = graph
+			.run(new org.apache.flink.graph.library.similarity.JaccardIndex<K, VV, EV>()
+				.setLittleParallelism(lp));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/PageRank.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/PageRank.java
new file mode 100644
index 0000000..8cef077
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/PageRank.java
@@ -0,0 +1,74 @@
+/*
+ * 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.flink.graph.drivers;
+
+import org.apache.commons.lang3.text.StrBuilder;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.drivers.output.CSV;
+import org.apache.flink.graph.drivers.output.Print;
+import org.apache.flink.graph.drivers.parameter.DoubleParameter;
+import org.apache.flink.graph.drivers.parameter.IterationConvergence;
+import org.apache.flink.graph.library.link_analysis.PageRank.Result;
+
+/**
+ * @see org.apache.flink.graph.library.link_analysis.PageRank
+ */
+public class PageRank<K, VV, EV>
+extends SimpleDriver<Result<K>>
+implements Driver<K, VV, EV>, CSV, Print {
+
+	private static final int DEFAULT_ITERATIONS = 10;
+
+	private DoubleParameter dampingFactor = new DoubleParameter(this, "damping_factor")
+		.setDefaultValue(0.85)
+		.setMinimumValue(0.0, false)
+		.setMaximumValue(1.0, false);
+
+	private IterationConvergence iterationConvergence = new IterationConvergence(this, DEFAULT_ITERATIONS);
+
+	@Override
+	public String getName() {
+		return this.getClass().getSimpleName();
+	}
+
+	@Override
+	public String getShortDescription() {
+		return "score vertices by the number and quality of incoming links";
+	}
+
+	@Override
+	public String getLongDescription() {
+		return new StrBuilder()
+			.appendln("PageRank computes a per-vertex score which is the sum of PageRank scores " +
+				"transmitted over in-edges. Each vertex's score is divided evenly among " +
+				"out-edges. High-scoring vertices are linked to by other high-scoring vertices.")
+			.appendNewLine()
+			.append("The result contains the vertex ID and PageRank score.")
+			.toString();
+	}
+
+	@Override
+	public void plan(Graph<K, VV, EV> graph) throws Exception {
+		result = graph
+			.run(new org.apache.flink.graph.library.link_analysis.PageRank<K, VV, EV>(
+				dampingFactor.getValue(),
+				iterationConvergence.getValue().iterations,
+				iterationConvergence.getValue().convergenceThreshold));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a48357db/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/SimpleDriver.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/SimpleDriver.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/SimpleDriver.java
new file mode 100644
index 0000000..98bdfc5
--- /dev/null
+++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/drivers/SimpleDriver.java
@@ -0,0 +1,65 @@
+/*
+ * 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.flink.graph.drivers;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode;
+import org.apache.flink.graph.asm.dataset.ChecksumHashCode.Checksum;
+import org.apache.flink.graph.asm.dataset.Collect;
+import org.apache.flink.graph.asm.result.PrintableResult;
+import org.apache.flink.graph.drivers.parameter.ParameterizedBase;
+
+import java.util.List;
+
+/**
+ * A base driver storing a single result {@link DataSet} with values
+ * implementing {@link PrintableResult}.
+ *
+ * @param <R> algorithm's result type
+ */
+public abstract class SimpleDriver<R extends PrintableResult>
+extends ParameterizedBase {
+
+	protected DataSet<? extends R> result;
+
+	public void hash(String executionName) throws Exception {
+		Checksum checksum = new ChecksumHashCode<R>()
+			.run((DataSet<R>) result)
+			.execute(executionName);
+
+		System.out.println(checksum);
+	}
+
+	public void print(String executionName) throws Exception {
+		Collect<R> collector = new Collect<>();
+
+		// Refactored due to openjdk7 compile error: https://travis-ci.org/greghogan/flink/builds/200487761
+		List<R> records = collector.run((DataSet<R>) result).execute(executionName);
+
+		for (R result : records) {
+			System.out.println(result.toPrintableString());
+		}
+	}
+
+	public void writeCSV(String filename, String lineDelimiter, String fieldDelimiter) {
+		result
+			.writeAsCsv(filename, lineDelimiter, fieldDelimiter)
+				.name("CSV: " + filename);
+	}
+}


[3/6] flink git commit: [FLINK-3695] [gelly] ValueArray types

Posted by gr...@apache.org.
[FLINK-3695] [gelly] ValueArray types

Provide compact and efficiently serializable and comparable array
implementations for Flink mutable Value types and Java primitives.

This cloeses #3382


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

Branch: refs/heads/master
Commit: 963f46e7179db034fd1d444469f4af58eac87409
Parents: 43158a8
Author: Greg Hogan <co...@greghogan.com>
Authored: Tue Feb 21 11:40:22 2017 -0500
Committer: Greg Hogan <co...@greghogan.com>
Committed: Fri Mar 31 11:14:36 2017 -0400

----------------------------------------------------------------------
 flink-libraries/flink-gelly/pom.xml             |  12 +-
 .../graph/types/valuearray/IntValueArray.java   | 398 ++++++++++++++
 .../valuearray/IntValueArrayComparator.java     | 156 ++++++
 .../valuearray/IntValueArraySerializer.java     |  85 +++
 .../graph/types/valuearray/LongValueArray.java  | 399 ++++++++++++++
 .../valuearray/LongValueArrayComparator.java    | 156 ++++++
 .../valuearray/LongValueArraySerializer.java    |  85 +++
 .../graph/types/valuearray/NullValueArray.java  | 267 ++++++++++
 .../valuearray/NullValueArrayComparator.java    | 147 ++++++
 .../valuearray/NullValueArraySerializer.java    |  85 +++
 .../types/valuearray/StringValueArray.java      | 518 +++++++++++++++++++
 .../valuearray/StringValueArrayComparator.java  | 217 ++++++++
 .../valuearray/StringValueArraySerializer.java  |  85 +++
 .../graph/types/valuearray/ValueArray.java      |  97 ++++
 .../types/valuearray/ValueArrayFactory.java     |  81 +++
 .../types/valuearray/ValueArrayTypeInfo.java    | 159 ++++++
 .../valuearray/ValueArrayTypeInfoFactory.java   |  41 ++
 .../valuearray/IntValueArrayComparatorTest.java |  51 ++
 .../valuearray/IntValueArraySerializerTest.java |  93 ++++
 .../types/valuearray/IntValueArrayTest.java     | 123 +++++
 .../LongValueArrayComparatorTest.java           |  51 ++
 .../LongValueArraySerializerTest.java           |  93 ++++
 .../types/valuearray/LongValueArrayTest.java    | 123 +++++
 .../NullValueArrayComparatorTest.java           |  51 ++
 .../NullValueArraySerializerTest.java           |  68 +++
 .../types/valuearray/NullValueArrayTest.java    |  80 +++
 .../StringValueArrayComparatorTest.java         |  51 ++
 .../StringValueArraySerializerTest.java         |  93 ++++
 .../types/valuearray/StringValueArrayTest.java  | 168 ++++++
 .../valuearray/ValueArrayTypeInfoTest.java      |  64 +++
 30 files changed, 4095 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml
index f773c70..fa09102 100644
--- a/flink-libraries/flink-gelly/pom.xml
+++ b/flink-libraries/flink-gelly/pom.xml
@@ -28,7 +28,7 @@ under the License.
 		<version>1.3-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
-	
+
 	<artifactId>flink-gelly_2.10</artifactId>
 	<name>flink-gelly</name>
 
@@ -59,7 +59,7 @@ under the License.
 		</dependency>
 
 		<!-- test dependencies -->
-		
+
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-test-utils_2.10</artifactId>
@@ -69,6 +69,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-optimizer_2.10</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArray.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArray.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArray.java
new file mode 100644
index 0000000..0e3812d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArray.java
@@ -0,0 +1,398 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.graph.utils.Murmur3_32;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * An array of {@link IntValue}.
+ */
+public class IntValueArray
+implements ValueArray<IntValue> {
+
+	protected static final int ELEMENT_LENGTH_IN_BYTES = 4;
+
+	protected static final int DEFAULT_CAPACITY_IN_BYTES = 4096;
+
+	// see note in ArrayList, HashTable, ...
+	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
+
+	private boolean isBounded;
+
+	private int[] data;
+
+	// the number of elements currently stored
+	private int position;
+
+	// location of the bookmark used by mark() and reset()
+	private transient int mark;
+
+	// hasher used to generate the normalized key
+	private Murmur3_32 hash = new Murmur3_32(0x11d2d865);
+
+	// hash result stored as normalized key
+	private IntValue hashValue = new IntValue();
+
+	/**
+	 * Initializes an expandable array with default capacity.
+	 */
+	public IntValueArray() {
+		isBounded = false;
+		initialize(DEFAULT_CAPACITY_IN_BYTES);
+	}
+
+	/**
+	 * Initializes a fixed-size array with the provided number of bytes.
+	 *
+	 * @param bytes number of bytes of the encapsulated array
+	 */
+	public IntValueArray(int bytes) {
+		isBounded = true;
+		initialize(bytes);
+	}
+
+	/**
+	 * Initializes the array with the provided number of bytes.
+	 *
+	 * @param bytes initial size of the encapsulated array in bytes
+	 */
+	private void initialize(int bytes) {
+		int capacity = bytes / ELEMENT_LENGTH_IN_BYTES;
+
+		Preconditions.checkArgument(capacity > 0, "Requested array with zero capacity");
+		Preconditions.checkArgument(capacity <= MAX_ARRAY_SIZE, "Requested capacity exceeds limit of " + MAX_ARRAY_SIZE);
+
+		data = new int[capacity];
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * If the size of the array is insufficient to hold the given capacity then
+	 * copy the array into a new, larger array.
+	 *
+	 * @param minCapacity minimum required number of elements
+	 */
+	private void ensureCapacity(int minCapacity) {
+		long currentCapacity = data.length;
+
+		if (minCapacity <= currentCapacity) {
+			return;
+		}
+
+		// increase capacity by at least ~50%
+		long expandedCapacity = Math.max(minCapacity, currentCapacity + (currentCapacity >> 1));
+		int newCapacity = (int) Math.min(MAX_ARRAY_SIZE, expandedCapacity);
+
+		if (newCapacity < minCapacity) {
+			// throw exception as unbounded arrays are not expected to fill
+			throw new RuntimeException("Requested array size " + minCapacity + " exceeds limit of " + MAX_ARRAY_SIZE);
+		}
+
+		data = Arrays.copyOf(data, newCapacity);
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder("[");
+		for (int idx = 0 ; idx < this.position ; idx++) {
+			sb.append(data[idx]);
+			if (idx < position - 1) {
+				sb.append(",");
+			}
+		}
+		sb.append("]");
+
+		return sb.toString();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Iterable
+	// --------------------------------------------------------------------------------------------
+
+	private final ReadIterator iterator = new ReadIterator();
+
+	@Override
+	public Iterator<IntValue> iterator() {
+		iterator.reset();
+		return iterator;
+	}
+
+	private class ReadIterator
+	implements Iterator<IntValue> {
+		private IntValue value = new IntValue();
+
+		private int pos;
+
+		@Override
+		public boolean hasNext() {
+			return pos < position;
+		}
+
+		@Override
+		public IntValue next() {
+			value.setValue(data[pos++]);
+			return value;
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException("remove");
+		}
+
+		public void reset() {
+			pos = 0;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// IOReadableWritable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		out.writeInt(position);
+
+		for (int i = 0 ; i < position ; i++) {
+			out.writeInt(data[i]);
+		}
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		position = in.readInt();
+		mark = 0;
+
+		ensureCapacity(position);
+
+		for (int i = 0 ; i < position ; i++) {
+			data[i] = in.readInt();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// NormalizableKey
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getMaxNormalizedKeyLen() {
+		return hashValue.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public void copyNormalizedKey(MemorySegment target, int offset, int len) {
+		hash.reset();
+
+		hash.hash(position);
+		for (int i = 0 ; i < position ; i++) {
+			hash.hash(data[i]);
+		}
+
+		hashValue.setValue(hash.hash());
+		hashValue.copyNormalizedKey(target, offset, len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Comparable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int compareTo(ValueArray<IntValue> o) {
+		IntValueArray other = (IntValueArray) o;
+
+		int min = Math.min(position, other.position);
+		for (int i = 0 ; i < min ; i++) {
+			int cmp = Integer.compare(data[i], other.data[i]);
+
+			if (cmp != 0) {
+				return cmp;
+			}
+		}
+
+		return Integer.compare(position, other.position);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Key
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		int hash = 1;
+
+		for (int i = 0 ; i < position ; i++) {
+			hash = 31 * hash + data[i];
+		}
+
+		return hash;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof IntValueArray) {
+			IntValueArray other = (IntValueArray) obj;
+
+			if (position != other.position) {
+				return false;
+			}
+
+			for (int i = 0 ; i < position ; i++) {
+				if (data[i] != other.data[i]) {
+					return false;
+				}
+			}
+
+			return true;
+		}
+
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ResettableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void setValue(ValueArray<IntValue> value) {
+		value.copyTo(this);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// CopyableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getBinaryLength() {
+		return -1;
+	}
+
+	@Override
+	public void copyTo(ValueArray<IntValue> target) {
+		IntValueArray other = (IntValueArray) target;
+
+		other.position = position;
+		other.mark = mark;
+
+		other.ensureCapacity(position);
+		System.arraycopy(data, 0, other.data, 0, position);
+	}
+
+	@Override
+	public ValueArray<IntValue> copy() {
+		ValueArray<IntValue> copy = new IntValueArray();
+
+		this.copyTo(copy);
+
+		return copy;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		copyInternal(source, target);
+	}
+
+	protected static void copyInternal(DataInputView source, DataOutputView target) throws IOException {
+		int count = source.readInt();
+		target.writeInt(count);
+
+		int bytes = ELEMENT_LENGTH_IN_BYTES * count;
+		target.write(source, bytes);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ValueArray
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int size() {
+		return position;
+	}
+
+	@Override
+	public boolean isFull() {
+		if (isBounded) {
+			return position == data.length;
+		} else {
+			return position == MAX_ARRAY_SIZE;
+		}
+	}
+
+	@Override
+	public boolean add(IntValue value) {
+		int newPosition = position + 1;
+
+		if (newPosition > data.length) {
+			if (isBounded) {
+				return false;
+			} else {
+				ensureCapacity(newPosition);
+			}
+		}
+
+		data[position] = value.getValue();
+		position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public boolean addAll(ValueArray<IntValue> other) {
+		IntValueArray source = (IntValueArray) other;
+
+		int sourceSize = source.position;
+		int newPosition = position + sourceSize;
+
+		if (newPosition > data.length) {
+			if (isBounded) {
+				return false;
+			} else {
+				ensureCapacity(newPosition);
+			}
+		}
+
+		System.arraycopy(source.data, 0, data, position, sourceSize);
+		position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public void clear() {
+		position = 0;
+	}
+
+	@Override
+	public void mark() {
+		mark = position;
+	}
+
+	@Override
+	public void reset() {
+		position = mark;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparator.java
new file mode 100644
index 0000000..bbc9bc5
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArrayComparator.java
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.NormalizableKey;
+
+import java.io.IOException;
+
+/**
+ * Specialized comparator for IntValueArray based on CopyableValueComparator.
+ *
+ * This can be used for grouping keys but not for sorting keys.
+ */
+@Internal
+public class IntValueArrayComparator extends TypeComparator<IntValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final boolean ascendingComparison;
+
+	private final IntValueArray reference = new IntValueArray();
+
+	private final TypeComparator<?>[] comparators = new TypeComparator[] {this};
+
+	public IntValueArrayComparator(boolean ascending) {
+		this.ascendingComparison = ascending;
+	}
+
+	@Override
+	public int hash(IntValueArray record) {
+		return record.hashCode();
+	}
+
+	@Override
+	public void setReference(IntValueArray toCompare) {
+		toCompare.copyTo(reference);
+	}
+
+	@Override
+	public boolean equalToReference(IntValueArray candidate) {
+		return candidate.equals(this.reference);
+	}
+
+	@Override
+	public int compareToReference(TypeComparator<IntValueArray> referencedComparator) {
+		int comp = ((IntValueArrayComparator) referencedComparator).reference.compareTo(reference);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compare(IntValueArray first, IntValueArray second) {
+		int comp = first.compareTo(second);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		int firstCount = firstSource.readInt();
+		int secondCount = secondSource.readInt();
+
+		int minCount = Math.min(firstCount, secondCount);
+		while (minCount-- > 0) {
+			int firstValue = firstSource.readInt();
+			int secondValue = secondSource.readInt();
+
+			int cmp = Integer.compare(firstValue, secondValue);
+			if (cmp != 0) {
+				return ascendingComparison ? cmp : -cmp;
+			}
+		}
+
+		int cmp = Integer.compare(firstCount, secondCount);
+		return ascendingComparison ? cmp : -cmp;
+	}
+
+	@Override
+	public boolean supportsNormalizedKey() {
+		return NormalizableKey.class.isAssignableFrom(IntValueArray.class);
+	}
+
+	@Override
+	public int getNormalizeKeyLen() {
+		return reference.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return keyBytes < getNormalizeKeyLen();
+	}
+
+	@Override
+	public void putNormalizedKey(IntValueArray record, MemorySegment target, int offset, int numBytes) {
+		record.copyNormalizedKey(target, offset, numBytes);
+	}
+
+	@Override
+	public boolean invertNormalizedKey() {
+		return !ascendingComparison;
+	}
+
+	@Override
+	public TypeComparator<IntValueArray> duplicate() {
+		return new IntValueArrayComparator(ascendingComparison);
+	}
+
+	@Override
+	public int extractKeys(Object record, Object[] target, int index) {
+		target[index] = record;
+		return 1;
+	}
+
+	@Override
+	public TypeComparator<?>[] getFlatComparators() {
+		return comparators;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// unsupported normalization
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		return false;
+	}
+
+	@Override
+	public void writeWithKeyNormalization(IntValueArray record, DataOutputView target) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public IntValueArray readWithKeyDenormalization(IntValueArray reuse, DataInputView source) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java
new file mode 100644
index 0000000..b86fe87
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Specialized serializer for {@code IntValueArray}.
+ */
+public final class IntValueArraySerializer extends TypeSerializerSingleton<IntValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public IntValueArray createInstance() {
+		return new IntValueArray();
+	}
+
+	@Override
+	public IntValueArray copy(IntValueArray from) {
+		return copy(from, new IntValueArray());
+	}
+
+	@Override
+	public IntValueArray copy(IntValueArray from, IntValueArray reuse) {
+		reuse.setValue(from);
+		return reuse;
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(IntValueArray record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+
+	@Override
+	public IntValueArray deserialize(DataInputView source) throws IOException {
+		return deserialize(new IntValueArray(), source);
+	}
+
+	@Override
+	public IntValueArray deserialize(IntValueArray reuse, DataInputView source) throws IOException {
+		reuse.read(source);
+		return reuse;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		IntValueArray.copyInternal(source, target);
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof IntValueArraySerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArray.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArray.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArray.java
new file mode 100644
index 0000000..7c01e6c
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArray.java
@@ -0,0 +1,399 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.graph.utils.Murmur3_32;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * An array of {@link LongValue}.
+ */
+public class LongValueArray
+implements ValueArray<LongValue> {
+
+	protected static final int ELEMENT_LENGTH_IN_BYTES = 8;
+
+	protected static final int DEFAULT_CAPACITY_IN_BYTES = 4096;
+
+	// see note in ArrayList, HashTable, ...
+	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
+
+	private boolean isBounded;
+
+	private long[] data;
+
+	// the number of elements currently stored
+	private int position;
+
+	// location of the bookmark used by mark() and reset()
+	private transient int mark;
+
+	// hasher used to generate the normalized key
+	private Murmur3_32 hash = new Murmur3_32(0xdf099ea8);
+
+	// hash result stored as normalized key
+	private IntValue hashValue = new IntValue();
+
+	/**
+	 * Initializes an expandable array with default capacity.
+	 */
+	public LongValueArray() {
+		isBounded = false;
+		initialize(DEFAULT_CAPACITY_IN_BYTES);
+	}
+
+	/**
+	 * Initializes a fixed-size array with the provided number of bytes.
+	 *
+	 * @param bytes number of bytes of the encapsulated array
+	 */
+	public LongValueArray(int bytes) {
+		isBounded = true;
+		initialize(bytes);
+	}
+
+	/**
+	 * Initializes the array with the provided number of bytes.
+	 *
+	 * @param bytes initial size of the encapsulated array in bytes
+	 */
+	private void initialize(int bytes) {
+		int capacity = bytes / ELEMENT_LENGTH_IN_BYTES;
+
+		Preconditions.checkArgument(capacity > 0, "Requested array with zero capacity");
+		Preconditions.checkArgument(capacity <= MAX_ARRAY_SIZE, "Requested capacity exceeds limit of " + MAX_ARRAY_SIZE);
+
+		data = new long[capacity];
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * If the size of the array is insufficient to hold the given capacity then
+	 * copy the array into a new, larger array.
+	 *
+	 * @param minCapacity minimum required number of elements
+	 */
+	private void ensureCapacity(int minCapacity) {
+		long currentCapacity = data.length;
+
+		if (minCapacity <= currentCapacity) {
+			return;
+		}
+
+		// increase capacity by at least ~50%
+		long expandedCapacity = Math.max(minCapacity, currentCapacity + (currentCapacity >> 1));
+		int newCapacity = (int) Math.min(MAX_ARRAY_SIZE, expandedCapacity);
+
+		if (newCapacity < minCapacity) {
+			// throw exception as unbounded arrays are not expected to fill
+			throw new RuntimeException("Requested array size " + minCapacity + " exceeds limit of " + MAX_ARRAY_SIZE);
+		}
+
+		data = Arrays.copyOf(data, newCapacity);
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder("[");
+		for (int idx = 0 ; idx < this.position ; idx++) {
+			sb.append(data[idx]);
+			if (idx < position - 1) {
+				sb.append(",");
+			}
+		}
+		sb.append("]");
+
+		return sb.toString();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Iterable
+	// --------------------------------------------------------------------------------------------
+
+	private final ReadIterator iterator = new ReadIterator();
+
+	@Override
+	public Iterator<LongValue> iterator() {
+		iterator.reset();
+		return iterator;
+	}
+
+	private class ReadIterator
+	implements Iterator<LongValue> {
+		private LongValue value = new LongValue();
+
+		private int pos;
+
+		@Override
+		public boolean hasNext() {
+			return pos < position;
+		}
+
+		@Override
+		public LongValue next() {
+			value.setValue(data[pos++]);
+			return value;
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException("remove");
+		}
+
+		public void reset() {
+			pos = 0;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// IOReadableWritable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		out.writeInt(position);
+
+		for (int i = 0 ; i < position ; i++) {
+			out.writeLong(data[i]);
+		}
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		position = in.readInt();
+		mark = 0;
+
+		ensureCapacity(position);
+
+		for (int i = 0 ; i < position ; i++) {
+			data[i] = in.readLong();
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// NormalizableKey
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getMaxNormalizedKeyLen() {
+		return hashValue.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public void copyNormalizedKey(MemorySegment target, int offset, int len) {
+		hash.reset();
+
+		hash.hash(position);
+		for (int i = 0 ; i < position ; i++) {
+			hash.hash(data[i]);
+		}
+
+		hashValue.setValue(hash.hash());
+		hashValue.copyNormalizedKey(target, offset, len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Comparable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int compareTo(ValueArray<LongValue> o) {
+		LongValueArray other = (LongValueArray) o;
+
+		int min = Math.min(position, other.position);
+		for (int i = 0 ; i < min ; i++) {
+			int cmp = Long.compare(data[i], other.data[i]);
+
+			if (cmp != 0) {
+				return cmp;
+			}
+		}
+
+		return Integer.compare(position, other.position);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Key
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		int hash = 1;
+
+		for (int i = 0 ; i < position ; i++) {
+			hash = 31 * hash + (int) (data[i] ^ data[i] >>> 32);
+		}
+
+		return hash;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof LongValueArray) {
+			LongValueArray other = (LongValueArray) obj;
+
+			if (position != other.position) {
+				return false;
+			}
+
+			for (int i = 0 ; i < position ; i++) {
+				if (data[i] != other.data[i]) {
+					return false;
+				}
+			}
+
+			return true;
+		}
+
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ResettableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void setValue(ValueArray<LongValue> value) {
+		value.copyTo(this);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// CopyableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getBinaryLength() {
+		return -1;
+	}
+
+	@Override
+	public void copyTo(ValueArray<LongValue> target) {
+		LongValueArray other = (LongValueArray) target;
+
+		other.position = position;
+		other.mark = mark;
+
+		other.ensureCapacity(position);
+		System.arraycopy(data, 0, other.data, 0, position);
+	}
+
+	@Override
+	public ValueArray<LongValue> copy() {
+		ValueArray<LongValue> copy = new LongValueArray();
+
+		this.copyTo(copy);
+
+		return copy;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		copyInternal(source, target);
+	}
+
+	protected static void copyInternal(DataInputView source, DataOutputView target) throws IOException {
+		int count = source.readInt();
+		target.writeInt(count);
+
+		int bytes = ELEMENT_LENGTH_IN_BYTES * count;
+		target.write(source, bytes);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ValueArray
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int size() {
+		return position;
+	}
+
+	@Override
+	public boolean isFull() {
+		if (isBounded) {
+			return position == data.length;
+		} else {
+			return position == MAX_ARRAY_SIZE;
+		}
+	}
+
+	@Override
+	public boolean add(LongValue value) {
+		int newPosition = position + 1;
+
+		if (newPosition > data.length) {
+			if (isBounded) {
+				return false;
+			} else {
+				ensureCapacity(newPosition);
+			}
+		}
+
+		data[position] = value.getValue();
+		position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public boolean addAll(ValueArray<LongValue> other) {
+		LongValueArray source = (LongValueArray) other;
+
+		int sourceSize = source.position;
+		int newPosition = position + sourceSize;
+
+		if (newPosition > data.length) {
+			if (isBounded) {
+				return false;
+			} else {
+				ensureCapacity(newPosition);
+			}
+		}
+
+		System.arraycopy(source.data, 0, data, position, sourceSize);
+		position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public void clear() {
+		position = 0;
+	}
+
+	@Override
+	public void mark() {
+		mark = position;
+	}
+
+	@Override
+	public void reset() {
+		position = mark;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparator.java
new file mode 100644
index 0000000..26c3da2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArrayComparator.java
@@ -0,0 +1,156 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.NormalizableKey;
+
+import java.io.IOException;
+
+/**
+ * Specialized comparator for LongValueArray based on CopyableValueComparator.
+ *
+ * This can be used for grouping keys but not for sorting keys.
+ */
+@Internal
+public class LongValueArrayComparator extends TypeComparator<LongValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final boolean ascendingComparison;
+
+	private final LongValueArray reference = new LongValueArray();
+
+	private final TypeComparator<?>[] comparators = new TypeComparator[] {this};
+
+	public LongValueArrayComparator(boolean ascending) {
+		this.ascendingComparison = ascending;
+	}
+
+	@Override
+	public int hash(LongValueArray record) {
+		return record.hashCode();
+	}
+
+	@Override
+	public void setReference(LongValueArray toCompare) {
+		toCompare.copyTo(reference);
+	}
+
+	@Override
+	public boolean equalToReference(LongValueArray candidate) {
+		return candidate.equals(this.reference);
+	}
+
+	@Override
+	public int compareToReference(TypeComparator<LongValueArray> referencedComparator) {
+		int comp = ((LongValueArrayComparator) referencedComparator).reference.compareTo(reference);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compare(LongValueArray first, LongValueArray second) {
+		int comp = first.compareTo(second);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		int firstCount = firstSource.readInt();
+		int secondCount = secondSource.readInt();
+
+		int minCount = Math.min(firstCount, secondCount);
+		while (minCount-- > 0) {
+			long firstValue = firstSource.readLong();
+			long secondValue = secondSource.readLong();
+
+			int cmp = Long.compare(firstValue, secondValue);
+			if (cmp != 0) {
+				return ascendingComparison ? cmp : -cmp;
+			}
+		}
+
+		int cmp = Integer.compare(firstCount, secondCount);
+		return ascendingComparison ? cmp : -cmp;
+	}
+
+	@Override
+	public boolean supportsNormalizedKey() {
+		return NormalizableKey.class.isAssignableFrom(LongValueArray.class);
+	}
+
+	@Override
+	public int getNormalizeKeyLen() {
+		return reference.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return keyBytes < getNormalizeKeyLen();
+	}
+
+	@Override
+	public void putNormalizedKey(LongValueArray record, MemorySegment target, int offset, int numBytes) {
+		record.copyNormalizedKey(target, offset, numBytes);
+	}
+
+	@Override
+	public boolean invertNormalizedKey() {
+		return !ascendingComparison;
+	}
+
+	@Override
+	public TypeComparator<LongValueArray> duplicate() {
+		return new LongValueArrayComparator(ascendingComparison);
+	}
+
+	@Override
+	public int extractKeys(Object record, Object[] target, int index) {
+		target[index] = record;
+		return 1;
+	}
+
+	@Override
+	public TypeComparator<?>[] getFlatComparators() {
+		return comparators;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// unsupported normalization
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		return false;
+	}
+
+	@Override
+	public void writeWithKeyNormalization(LongValueArray record, DataOutputView target) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public LongValueArray readWithKeyDenormalization(LongValueArray reuse, DataInputView source) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java
new file mode 100644
index 0000000..95219b6
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Specialized serializer for {@code LongValueArray}.
+ */
+public final class LongValueArraySerializer extends TypeSerializerSingleton<LongValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public LongValueArray createInstance() {
+		return new LongValueArray();
+	}
+
+	@Override
+	public LongValueArray copy(LongValueArray from) {
+		return copy(from, new LongValueArray());
+	}
+
+	@Override
+	public LongValueArray copy(LongValueArray from, LongValueArray reuse) {
+		reuse.setValue(from);
+		return reuse;
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(LongValueArray record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+
+	@Override
+	public LongValueArray deserialize(DataInputView source) throws IOException {
+		return deserialize(new LongValueArray(), source);
+	}
+
+	@Override
+	public LongValueArray deserialize(LongValueArray reuse, DataInputView source) throws IOException {
+		reuse.read(source);
+		return reuse;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		LongValueArray.copyInternal(source, target);
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof LongValueArraySerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArray.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArray.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArray.java
new file mode 100644
index 0000000..bf247a2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArray.java
@@ -0,0 +1,267 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.NullValue;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ * An array of {@link NullValue}.
+ */
+public class NullValueArray
+implements ValueArray<NullValue> {
+
+	// the number of elements currently stored
+	private int position;
+
+	// location of the bookmark used by mark() and reset()
+	private transient int mark;
+
+	// hash result stored as normalized key
+	private IntValue hashValue = new IntValue();
+
+	/**
+	 * Initializes an expandable array with default capacity.
+	 */
+	public NullValueArray() {
+	}
+
+	/**
+	 * Initializes a fixed-size array with the provided number of bytes.
+	 *
+	 * @param bytes number of bytes of the encapsulated array
+	 */
+	public NullValueArray(int bytes) {
+		this();
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder("[");
+		for (int idx = 0 ; idx < this.position ; idx++) {
+			sb.append("\u2205");
+			if (idx < position - 1) {
+				sb.append(",");
+			}
+		}
+		sb.append("]");
+
+		return sb.toString();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Iterable
+	// --------------------------------------------------------------------------------------------
+
+	private final ReadIterator iterator = new ReadIterator();
+
+	@Override
+	public Iterator<NullValue> iterator() {
+		iterator.reset();
+		return iterator;
+	}
+
+	private class ReadIterator
+	implements Iterator<NullValue> {
+		private int pos;
+
+		@Override
+		public boolean hasNext() {
+			return pos < position;
+		}
+
+		@Override
+		public NullValue next() {
+			pos++;
+			return NullValue.getInstance();
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException("remove");
+		}
+
+		public void reset() {
+			pos = 0;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// IOReadableWritable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		out.writeInt(position);
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		position = in.readInt();
+		mark = 0;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// NormalizableKey
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getMaxNormalizedKeyLen() {
+		return hashValue.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public void copyNormalizedKey(MemorySegment target, int offset, int len) {
+		hashValue.setValue(position);
+		hashValue.copyNormalizedKey(target, offset, len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Comparable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int compareTo(ValueArray<NullValue> o) {
+		NullValueArray other = (NullValueArray) o;
+
+		return Integer.compare(position, other.position);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Key
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return position;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof NullValueArray) {
+			NullValueArray other = (NullValueArray) obj;
+
+			return position == other.position;
+		}
+
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ResettableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void setValue(ValueArray<NullValue> value) {
+		value.copyTo(this);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// CopyableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getBinaryLength() {
+		return hashValue.getBinaryLength();
+	}
+
+	@Override
+	public void copyTo(ValueArray<NullValue> target) {
+		NullValueArray other = (NullValueArray) target;
+
+		other.position = position;
+	}
+
+	@Override
+	public ValueArray<NullValue> copy() {
+		ValueArray<NullValue> copy = new NullValueArray();
+
+		this.copyTo(copy);
+
+		return copy;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		target.write(source, getBinaryLength());
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ValueArray
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int size() {
+		return position;
+	}
+
+	@Override
+	public boolean isFull() {
+		return position == Integer.MAX_VALUE;
+	}
+
+	@Override
+	public boolean add(NullValue value) {
+		if (position == Integer.MAX_VALUE) {
+			return false;
+		}
+
+		position++;
+
+		return true;
+	}
+
+	@Override
+	public boolean addAll(ValueArray<NullValue> other) {
+		NullValueArray source = (NullValueArray) other;
+
+		long newPosition = position + (long) source.position;
+
+		if (newPosition > Integer.MAX_VALUE) {
+			return false;
+		}
+
+		position = (int) newPosition;
+
+		return true;
+	}
+
+	@Override
+	public void clear() {
+		position = 0;
+	}
+
+	@Override
+	public void mark() {
+		mark = position;
+	}
+
+	@Override
+	public void reset() {
+		position = mark;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparator.java
new file mode 100644
index 0000000..2228d6e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArrayComparator.java
@@ -0,0 +1,147 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.NormalizableKey;
+
+import java.io.IOException;
+
+/**
+ * Specialized comparator for NullValueArray based on CopyableValueComparator.
+ *
+ * This can be used for grouping keys but not for sorting keys.
+ */
+@Internal
+public class NullValueArrayComparator extends TypeComparator<NullValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final boolean ascendingComparison;
+
+	private final NullValueArray reference = new NullValueArray();
+
+	private final TypeComparator<?>[] comparators = new TypeComparator[] {this};
+
+	public NullValueArrayComparator(boolean ascending) {
+		this.ascendingComparison = ascending;
+	}
+
+	@Override
+	public int hash(NullValueArray record) {
+		return record.hashCode();
+	}
+
+	@Override
+	public void setReference(NullValueArray toCompare) {
+		toCompare.copyTo(reference);
+	}
+
+	@Override
+	public boolean equalToReference(NullValueArray candidate) {
+		return candidate.equals(this.reference);
+	}
+
+	@Override
+	public int compareToReference(TypeComparator<NullValueArray> referencedComparator) {
+		int comp = ((NullValueArrayComparator) referencedComparator).reference.compareTo(reference);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compare(NullValueArray first, NullValueArray second) {
+		int comp = first.compareTo(second);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		int firstCount = firstSource.readInt();
+		int secondCount = secondSource.readInt();
+
+		int cmp = Integer.compare(firstCount, secondCount);
+		return ascendingComparison ? cmp : -cmp;
+	}
+
+	@Override
+	public boolean supportsNormalizedKey() {
+		return NormalizableKey.class.isAssignableFrom(NullValueArray.class);
+	}
+
+	@Override
+	public int getNormalizeKeyLen() {
+		return reference.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return keyBytes < getNormalizeKeyLen();
+	}
+
+	@Override
+	public void putNormalizedKey(NullValueArray record, MemorySegment target, int offset, int numBytes) {
+		record.copyNormalizedKey(target, offset, numBytes);
+	}
+
+	@Override
+	public boolean invertNormalizedKey() {
+		return !ascendingComparison;
+	}
+
+	@Override
+	public TypeComparator<NullValueArray> duplicate() {
+		return new NullValueArrayComparator(ascendingComparison);
+	}
+
+	@Override
+	public int extractKeys(Object record, Object[] target, int index) {
+		target[index] = record;
+		return 1;
+	}
+
+	@Override
+	public TypeComparator<?>[] getFlatComparators() {
+		return comparators;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// key normalization
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		// see ComparatorTestBase#testNormalizedKeyReadWriter fixes in FLINK-4705
+		return false;
+	}
+
+	@Override
+	public void writeWithKeyNormalization(NullValueArray record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+
+	@Override
+	public NullValueArray readWithKeyDenormalization(NullValueArray reuse, DataInputView source) throws IOException {
+		reuse.read(source);
+		return reuse;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializer.java
new file mode 100644
index 0000000..233ed20
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Specialized serializer for {@code NullValueArray}.
+ */
+public final class NullValueArraySerializer extends TypeSerializerSingleton<NullValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public NullValueArray createInstance() {
+		return new NullValueArray();
+	}
+
+	@Override
+	public NullValueArray copy(NullValueArray from) {
+		return copy(from, new NullValueArray());
+	}
+
+	@Override
+	public NullValueArray copy(NullValueArray from, NullValueArray reuse) {
+		reuse.setValue(from);
+		return reuse;
+	}
+
+	@Override
+	public int getLength() {
+		return 4;
+	}
+
+	@Override
+	public void serialize(NullValueArray record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+
+	@Override
+	public NullValueArray deserialize(DataInputView source) throws IOException {
+		return deserialize(new NullValueArray(), source);
+	}
+
+	@Override
+	public NullValueArray deserialize(NullValueArray reuse, DataInputView source) throws IOException {
+		reuse.read(source);
+		return reuse;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		target.write(source, getLength());
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof NullValueArraySerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArray.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArray.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArray.java
new file mode 100644
index 0000000..4699552
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArray.java
@@ -0,0 +1,518 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.graph.utils.Murmur3_32;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * An array of {@link StringValue}.
+ * <p>
+ * Strings are serialized to a byte array. Concatenating arrays is as simple
+ * and fast as extending and copying byte arrays. Strings are serialized when
+ * individually added to {@code StringValueArray}.
+ * <p>
+ * For each string added to the array the length is first serialized using a
+ * variable length integer. Then the string characters are serialized using a
+ * variable length encoding where the lower 128 ASCII/UFT-8 characters are
+ * encoded in a single byte. This ensures that common characters are serialized
+ * in only two bytes.
+ */
+public class StringValueArray
+implements ValueArray<StringValue> {
+
+	protected static final int DEFAULT_CAPACITY_IN_BYTES = 4096;
+
+	// see note in ArrayList, HashTable, ...
+	private static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
+
+	protected static final int HIGH_BIT = 0x1 << 7;
+
+	private boolean isBounded;
+
+	// the initial length of a bounded array, which is allowed to expand to
+	// store one additional element beyond this initial length
+	private int boundedLength;
+
+	private byte[] data;
+
+	// number of StringValue elements currently stored
+	private int length;
+
+	// the number of bytes currently stored
+	private int position;
+
+	// state for the bookmark used by mark() and reset()
+	private transient int markLength;
+
+	private transient int markPosition;
+
+	// hasher used to generate the normalized key
+	private Murmur3_32 hash = new Murmur3_32(0x19264330);
+
+	// hash result stored as normalized key
+	private IntValue hashValue = new IntValue();
+
+	/**
+	 * Initializes an expandable array with default capacity.
+	 */
+	public StringValueArray() {
+		isBounded = false;
+		initialize(DEFAULT_CAPACITY_IN_BYTES);
+	}
+
+	/**
+	 * Initializes a fixed-size array with the provided number of bytes.
+	 *
+	 * @param bytes number of bytes of the encapsulated array
+	 */
+	public StringValueArray(int bytes) {
+		isBounded = true;
+		boundedLength = bytes;
+		initialize(bytes);
+	}
+
+	/**
+	 * Initializes the array with the provided number of bytes.
+	 *
+	 * @param bytes initial size of the encapsulated array in bytes
+	 */
+	private void initialize(int bytes) {
+		Preconditions.checkArgument(bytes > 0, "Requested array with zero capacity");
+		Preconditions.checkArgument(bytes <= MAX_ARRAY_SIZE, "Requested capacity exceeds limit of " + MAX_ARRAY_SIZE);
+
+		data = new byte[bytes];
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * If the size of the array is insufficient to hold the given capacity then
+	 * copy the array into a new, larger array.
+	 *
+	 * @param minCapacity minimum required number of elements
+	 */
+	private void ensureCapacity(int minCapacity) {
+		long currentCapacity = data.length;
+
+		if (minCapacity <= currentCapacity) {
+			return;
+		}
+
+		// increase capacity by at least ~50%
+		long expandedCapacity = Math.max(minCapacity, currentCapacity + (currentCapacity >> 1));
+		int newCapacity = (int) Math.min(MAX_ARRAY_SIZE, expandedCapacity);
+
+		if (newCapacity < minCapacity) {
+			// throw exception as unbounded arrays are not expected to fill
+			throw new RuntimeException("Requested array size " + minCapacity + " exceeds limit of " + MAX_ARRAY_SIZE);
+		}
+
+		data = Arrays.copyOf(data, newCapacity);
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder("[");
+		String separator = "";
+
+		for (StringValue sv : this) {
+			sb
+				.append(sv.getValue())
+				.append(separator);
+			separator = ",";
+		}
+
+		sb.append("]");
+
+		return sb.toString();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Iterable
+	// --------------------------------------------------------------------------------------------
+
+	private final ReadIterator iterator = new ReadIterator();
+
+	@Override
+	public Iterator<StringValue> iterator() {
+		iterator.reset();
+		return iterator;
+	}
+
+	private class ReadIterator
+	implements Iterator<StringValue> {
+		private static final int DEFAULT_SIZE = 64;
+
+		private StringValue value = new StringValue(CharBuffer.allocate(DEFAULT_SIZE));
+
+		private int size = DEFAULT_SIZE;
+
+		private int pos;
+
+		@Override
+		public boolean hasNext() {
+			return pos < position;
+		}
+
+		@Override
+		public StringValue next() {
+			// read length
+			int len = data[pos++] & 0xFF;
+
+			if (len >= HIGH_BIT) {
+				int shift = 7;
+				int curr;
+				len = len & 0x7F;
+				while ((curr = data[pos++] & 0xFF) >= HIGH_BIT) {
+					len |= (curr & 0x7F) << shift;
+					shift += 7;
+				}
+				len |= curr << shift;
+			}
+
+			// ensure capacity
+			if (len > size) {
+				while (size < len) {
+					size *= 2;
+				}
+
+				value = new StringValue(CharBuffer.allocate(size));
+			}
+
+			// read string characters
+			final char[] valueData = value.getCharArray();
+
+			for (int i = 0; i < len; i++) {
+				int c = data[pos++] & 0xFF;
+				if (c >= HIGH_BIT) {
+					int shift = 7;
+					int curr;
+					c = c & 0x7F;
+					while ((curr = data[pos++] & 0xFF) >= HIGH_BIT) {
+						c |= (curr & 0x7F) << shift;
+						shift += 7;
+					}
+					c |= curr << shift;
+				}
+				valueData[i] = (char) c;
+			}
+
+			return value;
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException("remove");
+		}
+
+		public void reset() {
+			pos = 0;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// IOReadableWritable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		out.writeInt(length);
+		out.writeInt(position);
+
+		out.write(data, 0, position);
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		length = in.readInt();
+		position = in.readInt();
+
+		markLength = 0;
+		markPosition = 0;
+
+		ensureCapacity(position);
+
+		in.read(data, 0, position);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// NormalizableKey
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getMaxNormalizedKeyLen() {
+		return hashValue.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public void copyNormalizedKey(MemorySegment target, int offset, int len) {
+		hash.reset();
+
+		hash.hash(position);
+		for (int i = 0 ; i < position ; i++) {
+			hash.hash(data[i]);
+		}
+
+		hashValue.setValue(hash.hash());
+		hashValue.copyNormalizedKey(target, offset, len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Comparable
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int compareTo(ValueArray<StringValue> o) {
+		StringValueArray other = (StringValueArray) o;
+
+		// sorts first on number of data in the array, then comparison between
+		// the first non-equal element in the arrays
+		int cmp = Integer.compare(position, other.position);
+
+		if (cmp != 0) {
+			return cmp;
+		}
+
+		for (int i = 0 ; i < position ; i++) {
+			cmp = Byte.compare(data[i], other.data[i]);
+
+			if (cmp != 0) {
+				return cmp;
+			}
+		}
+
+		return 0;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Key
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		int hash = 1;
+
+		for (int i = 0 ; i < position ; i++) {
+			hash = 31 * hash + data[i];
+		}
+
+		return hash;
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof StringValueArray) {
+			StringValueArray other = (StringValueArray) obj;
+
+			if (length != other.length) {
+				return false;
+			}
+
+			if (position != other.position) {
+				return false;
+			}
+
+			for (int i = 0 ; i < position ; i++) {
+				if (data[i] != other.data[i]) {
+					return false;
+				}
+			}
+
+			return true;
+		}
+
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ResettableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void setValue(ValueArray<StringValue> value) {
+		value.copyTo(this);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// CopyableValue
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int getBinaryLength() {
+		return -1;
+	}
+
+	@Override
+	public void copyTo(ValueArray<StringValue> target) {
+		StringValueArray other = (StringValueArray) target;
+
+		other.length = length;
+		other.position = position;
+		other.markLength = markLength;
+		other.markPosition = markPosition;
+
+		other.ensureCapacity(position);
+		System.arraycopy(data, 0, other.data, 0, position);
+	}
+
+	@Override
+	public ValueArray<StringValue> copy() {
+		ValueArray<StringValue> copy = new StringValueArray();
+
+		this.copyTo(copy);
+
+		return copy;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		copyInternal(source, target);
+	}
+
+	protected static void copyInternal(DataInputView source, DataOutputView target) throws IOException {
+		int length = source.readInt();
+		target.writeInt(length);
+
+		int position = source.readInt();
+		target.writeInt(position);
+
+		target.write(source, position);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// ValueArray
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int size() {
+		return length;
+	}
+
+	@Override
+	public boolean isFull() {
+		if (isBounded) {
+			return position >= boundedLength;
+		} else {
+			return position == MAX_ARRAY_SIZE;
+		}
+	}
+
+	@Override
+	public boolean add(StringValue value) {
+		if (isBounded && position >= boundedLength) {
+			return false;
+		}
+
+		// up to five bytes storing length
+		if (position + 5 > data.length) {
+			ensureCapacity(position + 5);
+		}
+
+		// update local variable until serialization succeeds
+		int newPosition = position;
+
+		// write the length, variable-length encoded
+		int len = value.length();
+
+		while (len >= HIGH_BIT) {
+			data[newPosition++] = (byte) (len | HIGH_BIT);
+			len >>>= 7;
+		}
+		data[newPosition++] = (byte) len;
+
+		// write the char data, variable-length encoded
+		final char[] valueData = value.getCharArray();
+		int remainingCapacity = data.length - newPosition;
+
+		len = value.length();
+		for (int i = 0; i < len; i++) {
+			// up to three bytes storing length
+			if (remainingCapacity < 3) {
+				ensureCapacity(remainingCapacity + 3);
+				remainingCapacity = data.length - newPosition;
+			}
+
+			int c = valueData[i];
+
+			while (c >= HIGH_BIT) {
+				data[newPosition++] = (byte) (c | HIGH_BIT);
+				remainingCapacity--;
+				c >>>= 7;
+			}
+			data[newPosition++] = (byte) c;
+			remainingCapacity--;
+		}
+
+		length++;
+		position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public boolean addAll(ValueArray<StringValue> other) {
+		StringValueArray source = (StringValueArray) other;
+
+		int sourceSize = source.position;
+		int newPosition = position + sourceSize;
+
+		if (newPosition > data.length) {
+			if (isBounded) {
+				return false;
+			} else {
+				ensureCapacity(newPosition);
+			}
+		}
+
+		System.arraycopy(source.data, 0, data, position, sourceSize);
+		length += source.length;
+  	    position = newPosition;
+
+		return true;
+	}
+
+	@Override
+	public void clear() {
+		length = 0;
+		position = 0;
+	}
+
+	@Override
+	public void mark() {
+		markLength = length;
+		markPosition = position;
+	}
+
+	@Override
+	public void reset() {
+		length = markLength;
+		position = markPosition;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparator.java
new file mode 100644
index 0000000..df88a8e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArrayComparator.java
@@ -0,0 +1,217 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.types.NormalizableKey;
+
+import java.io.IOException;
+
+import static org.apache.flink.graph.types.valuearray.StringValueArray.HIGH_BIT;
+
+/**
+ * Specialized comparator for StringValueArray based on CopyableValueComparator.
+ *
+ * This can be used for grouping keys but not for sorting keys.
+ */
+@Internal
+public class StringValueArrayComparator extends TypeComparator<StringValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final boolean ascendingComparison;
+
+	private final StringValueArray reference = new StringValueArray();
+
+	private final TypeComparator<?>[] comparators = new TypeComparator[] {this};
+
+	public StringValueArrayComparator(boolean ascending) {
+		this.ascendingComparison = ascending;
+	}
+
+	@Override
+	public int hash(StringValueArray record) {
+		return record.hashCode();
+	}
+
+	@Override
+	public void setReference(StringValueArray toCompare) {
+		toCompare.copyTo(reference);
+	}
+
+	@Override
+	public boolean equalToReference(StringValueArray candidate) {
+		return candidate.equals(this.reference);
+	}
+
+	@Override
+	public int compareToReference(TypeComparator<StringValueArray> referencedComparator) {
+		int comp = ((StringValueArrayComparator) referencedComparator).reference.compareTo(reference);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	@Override
+	public int compare(StringValueArray first, StringValueArray second) {
+		int comp = first.compareTo(second);
+		return ascendingComparison ? comp : -comp;
+	}
+
+	/**
+	 * Read the length of the next serialized {@code StringValue}.
+	 *
+	 * @param source the input view containing the record
+	 * @return the length of the next serialized {@code StringValue}
+	 * @throws IOException if the input view raised an exception when reading the length
+	 */
+	private static int readStringLength(DataInputView source) throws IOException {
+		int len = source.readByte() & 0xFF;
+
+		if (len >= HIGH_BIT) {
+			int shift = 7;
+			int curr;
+			len = len & 0x7F;
+			while ((curr = source.readByte() & 0xFF) >= HIGH_BIT) {
+				len |= (curr & 0x7F) << shift;
+				shift += 7;
+			}
+			len |= curr << shift;
+		}
+
+		return len;
+	}
+
+	/**
+	 * Read the next character from the serialized {@code StringValue}.
+	 *
+	 * @param source the input view containing the record
+	 * @return the next {@code char} of the current serialized {@code StringValue}
+	 * @throws IOException if the input view raised an exception when reading the length
+	 */
+	private static char readStringChar(DataInputView source) throws IOException {
+		int c = source.readByte() & 0xFF;
+
+		if (c >= HIGH_BIT) {
+			int shift = 7;
+			int curr;
+			c = c & 0x7F;
+			while ((curr = source.readByte() & 0xFF) >= HIGH_BIT) {
+				c |= (curr & 0x7F) << shift;
+				shift += 7;
+			}
+			c |= curr << shift;
+		}
+
+		return (char) c;
+	}
+
+	@Override
+	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+		int firstCount = firstSource.readInt();
+		int secondCount = secondSource.readInt();
+
+		int minCount = Math.min(firstCount, secondCount);
+		while (minCount-- > 0) {
+			int firstLength = readStringLength(firstSource);
+			int secondLength = readStringLength(secondSource);
+
+			int minLength = Math.min(firstLength, secondLength);
+			while (minLength-- > 0) {
+				char firstChar = readStringChar(firstSource);
+				char secondChar = readStringChar(secondSource);
+
+				int cmp = Character.compare(firstChar, secondChar);
+				if (cmp != 0) {
+					return ascendingComparison ? cmp : -cmp;
+				}
+			}
+
+			int cmp = Integer.compare(firstLength, secondLength);
+			if (cmp != 0) {
+				return ascendingComparison ? cmp : -cmp;
+			}
+		}
+
+		int cmp = Integer.compare(firstCount, secondCount);
+		return ascendingComparison ? cmp : -cmp;
+	}
+
+	@Override
+	public boolean supportsNormalizedKey() {
+		return NormalizableKey.class.isAssignableFrom(StringValueArray.class);
+	}
+
+	@Override
+	public int getNormalizeKeyLen() {
+		return reference.getMaxNormalizedKeyLen();
+	}
+
+	@Override
+	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+		return keyBytes < getNormalizeKeyLen();
+	}
+
+	@Override
+	public void putNormalizedKey(StringValueArray record, MemorySegment target, int offset, int numBytes) {
+		record.copyNormalizedKey(target, offset, numBytes);
+	}
+
+	@Override
+	public boolean invertNormalizedKey() {
+		return !ascendingComparison;
+	}
+
+	@Override
+	public TypeComparator<StringValueArray> duplicate() {
+		return new StringValueArrayComparator(ascendingComparison);
+	}
+
+	@Override
+	public int extractKeys(Object record, Object[] target, int index) {
+		target[index] = record;
+		return 1;
+	}
+
+	@Override
+	public TypeComparator<?>[] getFlatComparators() {
+		return comparators;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// unsupported normalization
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean supportsSerializationWithKeyNormalization() {
+		return false;
+	}
+
+	@Override
+	public void writeWithKeyNormalization(StringValueArray record, DataOutputView target) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public StringValueArray readWithKeyDenormalization(StringValueArray reuse, DataInputView source) throws IOException {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java
new file mode 100644
index 0000000..0e875e3
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Specialized serializer for {@code StringValueArray}.
+ */
+public final class StringValueArraySerializer extends TypeSerializerSingleton<StringValueArray> {
+
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public StringValueArray createInstance() {
+		return new StringValueArray();
+	}
+
+	@Override
+	public StringValueArray copy(StringValueArray from) {
+		return copy(from, new StringValueArray());
+	}
+
+	@Override
+	public StringValueArray copy(StringValueArray from, StringValueArray reuse) {
+		reuse.setValue(from);
+		return reuse;
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(StringValueArray record, DataOutputView target) throws IOException {
+		record.write(target);
+	}
+
+	@Override
+	public StringValueArray deserialize(DataInputView source) throws IOException {
+		return deserialize(new StringValueArray(), source);
+	}
+
+	@Override
+	public StringValueArray deserialize(StringValueArray reuse, DataInputView source) throws IOException {
+		reuse.read(source);
+		return reuse;
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		StringValueArray.copyInternal(source, target);
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof StringValueArraySerializer;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/963f46e7/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArray.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArray.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArray.java
new file mode 100644
index 0000000..6e34b71
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/ValueArray.java
@@ -0,0 +1,97 @@
+/*
+ * 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.flink.graph.types.valuearray;
+
+import org.apache.flink.api.common.typeinfo.TypeInfo;
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.types.CopyableValue;
+import org.apache.flink.types.NormalizableKey;
+import org.apache.flink.types.ResettableValue;
+import org.apache.flink.types.Value;
+
+import java.io.Serializable;
+
+/**
+ * Basic interface for array types which reuse objects during serialization.
+ *
+ * Value arrays are usable as grouping keys but not sorting keys.
+ *
+ * @param <T> the {@link Value} type
+ */
+@TypeInfo(ValueArrayTypeInfoFactory.class)
+public interface ValueArray<T>
+extends Iterable<T>, IOReadableWritable, Serializable, NormalizableKey<ValueArray<T>>, ResettableValue<ValueArray<T>>, CopyableValue<ValueArray<T>> {
+
+	/**
+	 * Returns the number of elements stored in the array.
+	 *
+	 * @return the number of elements stored in the array
+	 */
+	int size();
+
+	/**
+	 * An bounded array fills when the allocated capacity has been fully used.
+	 * An unbounded array will only fill when the underlying data structure has
+	 * reached capacity, for example the ~2^31 element limit for Java arrays.
+	 *
+	 * @return whether the array is full
+	 */
+	boolean isFull();
+
+	/**
+	 * Appends the value to this array if and only if the array capacity would
+	 * not be exceeded.
+	 *
+	 * @param value the value to add to this array
+	 * @return whether the value was added to the array
+	 */
+	boolean add(T value);
+
+	/**
+	 * Appends all of the values in the specified array to the end of this
+	 * array. If the combined array would exceed capacity then no values are
+	 * appended.
+	 *
+	 * @param source array containing values to be added to this array
+	 * @return whether the values were added to the array
+	 */
+	boolean addAll(ValueArray<T> source);
+
+	/**
+	 * Saves the array index, which can be restored by calling {@code reset()}.
+	 *
+	 * This is not serialized and is not part of the contract for
+	 * {@link #equals(Object)}.
+	 */
+	void mark();
+
+	/**
+	 * Restores the array index to when {@code mark()} was last called.
+	 */
+	void reset();
+
+	/**
+	 * Resets the array to the empty state. The implementation is *not*
+	 * expected to release the underlying data structure. This allows the array
+	 * to be reused with minimal impact on the garbage collector.
+	 *
+	 * This may reset the {@link #mark()} in order to allow arrays be shrunk.
+	 */
+	void clear();
+}