You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by va...@apache.org on 2015/10/09 18:05:39 UTC

[01/24] flink git commit: [FLINK-2784] Remove deprecated configuration keys and updated documentation

Repository: flink
Updated Branches:
  refs/heads/master 71d5a39b4 -> 3b77faa36


[FLINK-2784] Remove deprecated configuration keys and updated documentation

This closes #1244


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

Branch: refs/heads/master
Commit: 3b77faa36f11c704ebf69616c263dae3307e6ae7
Parents: 91ffbc1
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Oct 9 15:25:08 2015 +0200
Committer: vasia <va...@apache.org>
Committed: Fri Oct 9 17:33:15 2015 +0200

----------------------------------------------------------------------
 docs/apis/python.md                             |  2 +-
 docs/setup/config.md                            |  7 +--
 .../flink/configuration/ConfigConstants.java    | 29 ------------
 .../org/apache/flink/optimizer/Optimizer.java   |  7 +--
 .../minicluster/LocalFlinkMiniCluster.scala     | 19 ++------
 .../flink/runtime/taskmanager/TaskManager.scala | 48 ++++++--------------
 .../taskmanager/TaskManagerStartupTest.java     |  2 +-
 .../environment/StreamContextEnvironment.java   |  8 +---
 .../api/environment/StreamPlanEnvironment.java  |  8 +---
 9 files changed, 27 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/docs/apis/python.md
----------------------------------------------------------------------
diff --git a/docs/apis/python.md b/docs/apis/python.md
index 14585fc..d57e117 100644
--- a/docs/apis/python.md
+++ b/docs/apis/python.md
@@ -575,7 +575,7 @@ env.execute()
 ### System Level
 
 A system-wide default parallelism for all execution environments can be defined by setting the
-`parallelization.degree.default` property in `./conf/flink-conf.yaml`. See the
+`parallelism.default` property in `./conf/flink-conf.yaml`. See the
 [Configuration](config.html) documentation for details.
 
 [Back to top](#top)

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index 4cfe428..0e7b2ee 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -142,6 +142,9 @@ results outside of the JVM heap. For setups with larger quantities of memory,
 this can improve the efficiency of the operations performed on the memory
 (DEFAULT: false).
 
+- `taskmanager.memory.segment-size`: The size of memory buffers used by the 
+memory manager and the network stack in bytes (DEFAULT: 32768 (= 32 KiBytes)).
+
 ### Other
 
 - `taskmanager.tmp.dirs`: The directory for temporary files, or a list of
@@ -254,8 +257,6 @@ network stack. This number determines how many streaming data exchange channels
 a TaskManager can have at the same time and how well buffered the channels are.
 If a job is rejected or you get a warning that the system has not enough buffers
 available, increase this value (DEFAULT: 2048).
-- `taskmanager.network.bufferSizeInBytes`: The size of the network buffers, in
-bytes (DEFAULT: 32768 (= 32 KiBytes)).
 - `taskmanager.memory.size`: The amount of memory (in megabytes) that the task
 manager reserves on the JVM's heap space for sorting, hash tables, and caching
 of intermediate results. If unspecified (-1), the memory manager will take a fixed
@@ -441,7 +442,7 @@ The number and size of network buffers can be configured with the following
 parameters:
 
 - `taskmanager.network.numberOfBuffers`, and
-- `taskmanager.network.bufferSizeInBytes`.
+- `taskmanager.memory.segment-size`.
 
 ### Configuring Temporary I/O Directories
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 2b4749c..36369ab 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -36,12 +36,6 @@ public final class ConfigConstants {
 	public static final String DEFAULT_PARALLELISM_KEY = "parallelism.default";
 
 	/**
-	 * The deprecated config parameter defining the default parallelism for jobs.
-	 */
-	@Deprecated
-	public static final String DEFAULT_PARALLELISM_KEY_OLD = "parallelization.degree.default";
-
-	/**
 	 * Config parameter for the number of re-tries for failed tasks. Setting this
 	 * value to 0 effectively disables fault tolerance.
 	 */
@@ -136,12 +130,6 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY = "taskmanager.network.numberOfBuffers";
 
 	/**
-	 * Deprecated config parameter defining the size of the buffers used in the network stack.
-	 */
-	@Deprecated
-	public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
-
-	/**
 	 * Config parameter defining the size of memory buffers used by the network stack and the memory manager.
 	 */
 	public static final String TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY = "taskmanager.memory.segment-size";
@@ -206,21 +194,10 @@ public final class ConfigConstants {
 	public static final String YARN_HEAP_CUTOFF_RATIO = "yarn.heap-cutoff-ratio";
 
 	/**
-	 * Upper bound for heap cutoff on YARN.
-	 * The "yarn.heap-cutoff-ratio" is removing a certain ratio from the heap.
-	 * This value is limiting this cutoff to a absolute value.
-	 *
-	 * THE VALUE IS NO LONGER IN USE.
-	 */
-	@Deprecated
-	public static final String YARN_HEAP_LIMIT_CAP = "yarn.heap-limit-cap";
-
-	/**
 	 * Minimum amount of memory to remove from the heap space as a safety margin.
 	 */
 	public static final String YARN_HEAP_CUTOFF_MIN = "yarn.heap-cutoff-min";
 
-
 	/**
 	 * Reallocate failed YARN containers.
 	 */
@@ -548,12 +525,6 @@ public final class ConfigConstants {
 	public static final int DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS = 2048;
 
 	/**
-	 * Default size of network stack buffers.
-	 */
-	@Deprecated
-	public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768;
-
-	/**
 	 * Default size of memory segments in the network stack and the memory manager.
 	 */
 	public static final int DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE = 32768;

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
index 6f41c29..ed3cbd5 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/Optimizer.java
@@ -347,14 +347,9 @@ public class Optimizer {
 		this.costEstimator = estimator;
 
 		// determine the default parallelism
-		// check for old key string first, then for new one
-		this.defaultParallelism = config.getInteger(
-				ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
-				ConfigConstants.DEFAULT_PARALLELISM);
-		// now check for new one which overwrites old values
 		this.defaultParallelism = config.getInteger(
 				ConfigConstants.DEFAULT_PARALLELISM_KEY,
-				this.defaultParallelism);
+				ConfigConstants.DEFAULT_PARALLELISM);
 
 		if (defaultParallelism < 1) {
 			LOG.warn("Config value " + defaultParallelism + " for option "

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
index 7b1f9e8..38e3efb 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala
@@ -161,22 +161,9 @@ class LocalFlinkMiniCluster(
     // set this only if no memory was pre-configured
     if (config.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1) == -1) {
 
-      val bufferSizeNew: Int = config.getInteger(
-        ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, -1)
-
-      val bufferSizeOld: Int = config.getInteger(
-        ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, -1)
-      val bufferSize: Int =
-        if (bufferSizeNew != -1) {
-          bufferSizeNew
-        }
-        else if (bufferSizeOld == -1) {
-          // nothing has been configured, take the default
-          ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE
-        }
-        else {
-          bufferSizeOld
-        }
+      val bufferSize: Int = config.getInteger(
+        ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+        ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE)
       
       val bufferMem: Long = config.getLong(
         ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index bf23021..b28fb73 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -1758,41 +1758,19 @@ object TaskManager {
     checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
       ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY)
     
-    val pageSizeNew: Int = configuration.getInteger(
-      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, -1)
-    
-    val pageSizeOld: Int = configuration.getInteger(
-      ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, -1)
-
-    val pageSize: Int =
-      if (pageSizeNew != -1) {
-        // new page size has been configured
-        checkConfigParameter(pageSizeNew >= MemoryManager.MIN_PAGE_SIZE, pageSizeNew,
-          ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-          "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE)
-
-        checkConfigParameter(MathUtils.isPowerOf2(pageSizeNew), pageSizeNew,
-          ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-          "Memory segment size must be a power of 2.")
-
-        pageSizeNew
-      }
-      else if (pageSizeOld == -1) {
-        // nothing has been configured, take the default
-        ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE
-      }
-      else {
-        // old page size has been configured
-        checkConfigParameter(pageSizeOld >= MemoryManager.MIN_PAGE_SIZE, pageSizeOld,
-          ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-          "Minimum buffer size is " + MemoryManager.MIN_PAGE_SIZE)
-
-        checkConfigParameter(MathUtils.isPowerOf2(pageSizeOld), pageSizeOld,
-          ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-          "Buffer size must be a power of 2.")
-
-        pageSizeOld
-      }
+    val pageSize: Int = configuration.getInteger(
+      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+      ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE)
+
+    // check page size of for minimum size
+    checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+      "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE)
+
+    // check page size for power of two
+    checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+      ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+      "Memory segment size must be a power of 2.")
     
     // check whether we use heap or off-heap memory
     val memType: MemoryType = 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
index a3b65f0..9cc8170 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java
@@ -149,7 +149,7 @@ public class TaskManagerStartupTest {
 
 			// something ridiculously high
 			final long memSize = (((long) Integer.MAX_VALUE - 1) *
-									ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE) >> 20;
+									ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE) >> 20;
 			cfg.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memSize);
 			try {
 				TaskManager.runTaskManager("localhost", 0, cfg, StreamingMode.BATCH_ONLY);

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
index 1392efb..b2a5435 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
@@ -60,14 +60,10 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
 			setParallelism(parallelism);
 		}
 		else {
-			// first check for old parallelism config key
-			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
-					ConfigConstants.DEFAULT_PARALLELISM));
-			// then for new
+			// determine parallelism
 			setParallelism(GlobalConfiguration.getInteger(
 					ConfigConstants.DEFAULT_PARALLELISM_KEY,
-					getParallelism()));
+					ConfigConstants.DEFAULT_PARALLELISM));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3b77faa3/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
index e5ea2c5..61b9a2f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
@@ -37,14 +37,10 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment {
 		if (parallelism > 0) {
 			setParallelism(parallelism);
 		} else {
-			// first check for old parallelism config key
-			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY_OLD,
-					ConfigConstants.DEFAULT_PARALLELISM));
-			// then for new
+			// determine parallelism
 			setParallelism(GlobalConfiguration.getInteger(
 					ConfigConstants.DEFAULT_PARALLELISM_KEY,
-					getParallelism()));
+					ConfigConstants.DEFAULT_PARALLELISM));
 		}
 	}
 


[06/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
deleted file mode 100644
index afd4ffd..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
+++ /dev/null
@@ -1,135 +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.spargel;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.IterationConfiguration;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A VertexCentricConfiguration object can be used to set the iteration name and
- * degree of parallelism, to register aggregators and use broadcast sets in
- * the {@link org.apache.flink.graph.spargel.VertexUpdateFunction} and {@link org.apache.flink.graph.spargel.MessagingFunction}
- *
- * The VertexCentricConfiguration object is passed as an argument to
- * {@link org.apache.flink.graph.Graph#runVertexCentricIteration (
- * org.apache.flink.graph.spargel.VertexUpdateFunction, org.apache.flink.graph.spargel.MessagingFunction, int,
- * VertexCentricConfiguration)}.
- */
-public class VertexCentricConfiguration extends IterationConfiguration {
-
-	/** the broadcast variables for the update function **/
-	private List<Tuple2<String, DataSet<?>>> bcVarsUpdate = new ArrayList<Tuple2<String,DataSet<?>>>();
-
-	/** the broadcast variables for the messaging function **/
-	private List<Tuple2<String, DataSet<?>>> bcVarsMessaging = new ArrayList<Tuple2<String,DataSet<?>>>();
-
-	/** flag that defines whether the degrees option is set **/
-	private boolean optDegrees = false;
-
-	/** the direction in which the messages should be sent **/
-	private EdgeDirection direction = EdgeDirection.OUT;
-
-	public VertexCentricConfiguration() {}
-
-	/**
-	 * Adds a data set as a broadcast set to the messaging function.
-	 *
-	 * @param name The name under which the broadcast data is available in the messaging function.
-	 * @param data The data set to be broadcasted.
-	 */
-	public void addBroadcastSetForMessagingFunction(String name, DataSet<?> data) {
-		this.bcVarsMessaging.add(new Tuple2<String, DataSet<?>>(name, data));
-	}
-
-	/**
-	 * Adds a data set as a broadcast set to the vertex update function.
-	 *
-	 * @param name The name under which the broadcast data is available in the vertex update function.
-	 * @param data The data set to be broadcasted.
-	 */
-	public void addBroadcastSetForUpdateFunction(String name, DataSet<?> data) {
-		this.bcVarsUpdate.add(new Tuple2<String, DataSet<?>>(name, data));
-	}
-
-	/**
-	 * Get the broadcast variables of the VertexUpdateFunction.
-	 *
-	 * @return a List of Tuple2, where the first field is the broadcast variable name
-	 * and the second field is the broadcast DataSet.
-	 */
-	public List<Tuple2<String, DataSet<?>>> getUpdateBcastVars() {
-		return this.bcVarsUpdate;
-	}
-
-	/**
-	 * Get the broadcast variables of the MessagingFunction.
-	 *
-	 * @return a List of Tuple2, where the first field is the broadcast variable name
-	 * and the second field is the broadcast DataSet.
-	 */
-	public List<Tuple2<String, DataSet<?>>> getMessagingBcastVars() {
-		return this.bcVarsMessaging;
-	}
-
-	/**
-	 * Gets whether the degrees option is set.
-	 * By default, the degrees option is not set.
-	 *
-	 * @return True, if the degree option is set, false otherwise.
-	 */
-	public boolean isOptDegrees() {
-		return optDegrees;
-	}
-
-	/**
-	 * Sets the degree option.
-	 * By default, the degrees option is not set.
-	 *
-	 * @param optDegrees True, to set this option, false otherwise.
-	 */
-	public void setOptDegrees(boolean optDegrees) {
-		this.optDegrees = optDegrees;
-	}
-
-	/**
-	 * Gets the direction in which messages are sent in the MessagingFunction.
-	 * By default the messaging direction is OUT.
-	 *
-	 * @return an EdgeDirection, which can be either IN, OUT or ALL.
-	 */
-	public EdgeDirection getDirection() {
-		return direction;
-	}
-
-	/**
-	 * Sets the direction in which messages are sent in the MessagingFunction.
-	 * By default the messaging direction is OUT.
-	 *
-	 * @param direction - IN, OUT or ALL
-	 */
-	public void setDirection(EdgeDirection direction) {
-		this.direction = direction;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
deleted file mode 100644
index b3a470e..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
+++ /dev/null
@@ -1,686 +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.spargel;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.common.functions.RichCoGroupFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.operators.CoGroupOperator;
-import org.apache.flink.api.java.operators.CustomUnaryOperation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.util.Collector;
-
-import com.google.common.base.Preconditions;
-
-/**
- * This class represents iterative graph computations, programmed in a vertex-centric perspective.
- * It is a special case of <i>Bulk Synchronous Parallel<i> computation. The paradigm has also been
- * implemented by Google's <i>Pregel</i> system and by <i>Apache Giraph</i>.
- * <p>
- * Vertex centric algorithms operate on graphs, which are defined through vertices and edges. The 
- * algorithms send messages along the edges and update the state of vertices based on
- * the old state and the incoming messages. All vertices have an initial state.
- * The computation terminates once no vertex updates it state any more.
- * Additionally, a maximum number of iterations (supersteps) may be specified.
- * <p>
- * The computation is here represented by two functions:
- * <ul>
- *   <li>The {@link VertexUpdateFunction} receives incoming messages and may updates the state for
- *   the vertex. If a state is updated, messages are sent from this vertex. Initially, all vertices are
- *   considered updated.</li>
- *   <li>The {@link MessagingFunction} takes the new vertex state and sends messages along the outgoing
- *   edges of the vertex. The outgoing edges may optionally have an associated value, such as a weight.</li>
- * </ul>
- * <p>
- *
- * Vertex-centric graph iterations are are run by calling
- * {@link Graph#runVertexCentricIteration(VertexUpdateFunction, MessagingFunction, int)}.
- *
- * @param <K> The type of the vertex key (the vertex identifier).
- * @param <VV> The type of the vertex value (the state of the vertex).
- * @param <Message> The type of the message sent between vertices along the edges.
- * @param <EV> The type of the values that are associated with the edges.
- */
-public class VertexCentricIteration<K, VV, Message, EV> 
-	implements CustomUnaryOperation<Vertex<K, VV>, Vertex<K, VV>>
-{
-	private final VertexUpdateFunction<K, VV, Message> updateFunction;
-
-	private final MessagingFunction<K, VV, Message, EV> messagingFunction;
-	
-	private final DataSet<Edge<K, EV>> edgesWithValue;
-	
-	private final int maximumNumberOfIterations;
-	
-	private final TypeInformation<Message> messageType;
-	
-	private DataSet<Vertex<K, VV>> initialVertices;
-
-	private VertexCentricConfiguration configuration;
-
-	// ----------------------------------------------------------------------------------
-	
-	private VertexCentricIteration(VertexUpdateFunction<K, VV, Message> uf,
-			MessagingFunction<K, VV, Message, EV> mf,
-			DataSet<Edge<K, EV>> edgesWithValue, 
-			int maximumNumberOfIterations)
-	{
-		Preconditions.checkNotNull(uf);
-		Preconditions.checkNotNull(mf);
-		Preconditions.checkNotNull(edgesWithValue);
-		Preconditions.checkArgument(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one.");
-
-		this.updateFunction = uf;
-		this.messagingFunction = mf;
-		this.edgesWithValue = edgesWithValue;
-		this.maximumNumberOfIterations = maximumNumberOfIterations;		
-		this.messageType = getMessageType(mf);
-	}
-	
-	private TypeInformation<Message> getMessageType(MessagingFunction<K, VV, Message, EV> mf) {
-		return TypeExtractor.createTypeInfo(MessagingFunction.class, mf.getClass(), 2, null, null);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Custom Operator behavior
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Sets the input data set for this operator. In the case of this operator this input data set represents
-	 * the set of vertices with their initial state.
-	 * 
-	 * @param inputData The input data set, which in the case of this operator represents the set of
-	 *                  vertices with their initial state.
-	 * 
-	 * @see org.apache.flink.api.java.operators.CustomUnaryOperation#setInput(org.apache.flink.api.java.DataSet)
-	 */
-	@Override
-	public void setInput(DataSet<Vertex<K, VV>> inputData) {
-		this.initialVertices = inputData;
-	}
-	
-	/**
-	 * Creates the operator that represents this vertex-centric graph computation.
-	 * 
-	 * @return The operator that represents this vertex-centric graph computation.
-	 */
-	@Override
-	public DataSet<Vertex<K, VV>> createResult() {
-		if (this.initialVertices == null) {
-			throw new IllegalStateException("The input data set has not been set.");
-		}
-
-		// prepare some type information
-		TypeInformation<K> keyType = ((TupleTypeInfo<?>) initialVertices.getType()).getTypeAt(0);
-		TypeInformation<Tuple2<K, Message>> messageTypeInfo = new TupleTypeInfo<Tuple2<K,Message>>(keyType, messageType);
-
-		// create a graph
-		Graph<K, VV, EV> graph =
-				Graph.fromDataSet(initialVertices, edgesWithValue, initialVertices.getExecutionEnvironment());
-
-		// check whether the numVertices option is set and, if so, compute the total number of vertices
-		// and set it within the messaging and update functions
-
-		if (this.configuration != null && this.configuration.isOptNumVertices()) {
-			try {
-				long numberOfVertices = graph.numberOfVertices();
-				messagingFunction.setNumberOfVertices(numberOfVertices);
-				updateFunction.setNumberOfVertices(numberOfVertices);
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
-
-		if(this.configuration != null) {
-			messagingFunction.setDirection(this.configuration.getDirection());
-		} else {
-			messagingFunction.setDirection(EdgeDirection.OUT);
-		}
-
-		// retrieve the direction in which the updates are made and in which the messages are sent
-		EdgeDirection messagingDirection = messagingFunction.getDirection();
-
-		// check whether the degrees option is set and, if so, compute the in and the out degrees and
-		// add them to the vertex value
-		if(this.configuration != null && this.configuration.isOptDegrees()) {
-			return createResultVerticesWithDegrees(graph, messagingDirection, messageTypeInfo);
-		} else {
-			return createResultSimpleVertex(messagingDirection, messageTypeInfo);
-		}
-	}
-
-	/**
-	 * Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as
-	 * a weight or distance).
-	 * 
-	 * @param edgesWithValue The data set containing edges.
-	 * @param uf The function that updates the state of the vertices from the incoming messages.
-	 * @param mf The function that turns changed vertex states into messages along the edges.
-	 * 
-	 * @param <K> The type of the vertex key (the vertex identifier).
-	 * @param <VV> The type of the vertex value (the state of the vertex).
-	 * @param <Message> The type of the message sent between vertices along the edges.
-	 * @param <EV> The type of the values that are associated with the edges.
-	 * 
-	 * @return An in stance of the vertex-centric graph computation operator.
-	 */
-	public static final <K, VV, Message, EV>
-			VertexCentricIteration<K, VV, Message, EV> withEdges(
-					DataSet<Edge<K, EV>> edgesWithValue,
-					VertexUpdateFunction<K, VV, Message> uf,
-					MessagingFunction<K, VV, Message, EV> mf,
-					int maximumNumberOfIterations)
-	{
-		return new VertexCentricIteration<K, VV, Message, EV>(uf, mf, edgesWithValue, maximumNumberOfIterations);
-	}
-
-	/**
-	 * Configures this vertex-centric iteration with the provided parameters.
-	 *
-	 * @param parameters the configuration parameters
-	 */
-	public void configure(VertexCentricConfiguration parameters) {
-		this.configuration = parameters;
-	}
-
-	/**
-	 * @return the configuration parameters of this vertex-centric iteration
-	 */
-	public VertexCentricConfiguration getIterationConfiguration() {
-		return this.configuration;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Wrapping UDFs
-	// --------------------------------------------------------------------------------------------
-
-	private static abstract class VertexUpdateUdf<K, VVWithDegrees, Message> extends RichCoGroupFunction<
-		Tuple2<K, Message>, Vertex<K, VVWithDegrees>, Vertex<K, VVWithDegrees>>
-		implements ResultTypeQueryable<Vertex<K, VVWithDegrees>>
-	{
-		private static final long serialVersionUID = 1L;
-		
-		final VertexUpdateFunction<K, VVWithDegrees, Message> vertexUpdateFunction;
-
-		final MessageIterator<Message> messageIter = new MessageIterator<Message>();
-		
-		private transient TypeInformation<Vertex<K, VVWithDegrees>> resultType;
-		
-		
-		private VertexUpdateUdf(VertexUpdateFunction<K, VVWithDegrees, Message> vertexUpdateFunction,
-				TypeInformation<Vertex<K, VVWithDegrees>> resultType)
-		{
-			this.vertexUpdateFunction = vertexUpdateFunction;
-			this.resultType = resultType;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
-				this.vertexUpdateFunction.init(getIterationRuntimeContext());
-			}
-			this.vertexUpdateFunction.preSuperstep();
-		}
-		
-		@Override
-		public void close() throws Exception {
-			this.vertexUpdateFunction.postSuperstep();
-		}
-
-		@Override
-		public TypeInformation<Vertex<K, VVWithDegrees>> getProducedType() {
-			return this.resultType;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class VertexUpdateUdfSimpleVV<K, VV, Message> extends VertexUpdateUdf<K, VV, Message> {
-
-		private VertexUpdateUdfSimpleVV(VertexUpdateFunction<K, VV, Message> vertexUpdateFunction, TypeInformation<Vertex<K, VV>> resultType) {
-			super(vertexUpdateFunction, resultType);
-		}
-
-		@Override
-		public void coGroup(Iterable<Tuple2<K, Message>> messages,
-							Iterable<Vertex<K, VV>> vertex,
-							Collector<Vertex<K, VV>> out) throws Exception {
-			final Iterator<Vertex<K, VV>> vertexIter = vertex.iterator();
-
-			if (vertexIter.hasNext()) {
-				Vertex<K, VV> vertexState = vertexIter.next();
-
-				@SuppressWarnings("unchecked")
-				Iterator<Tuple2<?, Message>> downcastIter = (Iterator<Tuple2<?, Message>>) (Iterator<?>) messages.iterator();
-				messageIter.setSource(downcastIter);
-
-				vertexUpdateFunction.setOutput(vertexState, out);
-				vertexUpdateFunction.updateVertex(vertexState, messageIter);
-			}
-			else {
-				final Iterator<Tuple2<K, Message>> messageIter = messages.iterator();
-				if (messageIter.hasNext()) {
-					String message = "Target vertex does not exist!.";
-					try {
-						Tuple2<K, Message> next = messageIter.next();
-						message = "Target vertex '" + next.f0 + "' does not exist!.";
-					} catch (Throwable t) {}
-					throw new Exception(message);
-				} else {
-					throw new Exception();
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class VertexUpdateUdfVVWithDegrees<K, VV, Message> extends VertexUpdateUdf<K, Tuple3<VV, Long, Long>, Message> {
-
-		private VertexUpdateUdfVVWithDegrees(VertexUpdateFunction<K, Tuple3<VV, Long, Long>, Message> vertexUpdateFunction,
-				TypeInformation<Vertex<K, Tuple3<VV, Long, Long>>> resultType) {
-			super(vertexUpdateFunction, resultType);
-		}
-		
-		@Override
-		public void coGroup(Iterable<Tuple2<K, Message>> messages, Iterable<Vertex<K, Tuple3<VV, Long, Long>>> vertex,
-							Collector<Vertex<K, Tuple3<VV, Long, Long>>> out) throws Exception {
-
-			final Iterator<Vertex<K, Tuple3<VV, Long, Long>>> vertexIter = vertex.iterator();
-		
-			if (vertexIter.hasNext()) {
-				Vertex<K, Tuple3<VV, Long, Long>> vertexWithDegrees = vertexIter.next();
-		
-				@SuppressWarnings("unchecked")
-				Iterator<Tuple2<?, Message>> downcastIter = (Iterator<Tuple2<?, Message>>) (Iterator<?>) messages.iterator();
-				messageIter.setSource(downcastIter);
-
-				vertexUpdateFunction.setInDegree(vertexWithDegrees.f1.f1);
-				vertexUpdateFunction.setOutDegree(vertexWithDegrees.f1.f2);
-
-				vertexUpdateFunction.setOutputWithDegrees(vertexWithDegrees, out);
-				vertexUpdateFunction.updateVertexFromVertexCentricIteration(vertexWithDegrees, messageIter);
-			}
-			else {
-				final Iterator<Tuple2<K, Message>> messageIter = messages.iterator();
-				if (messageIter.hasNext()) {
-					String message = "Target vertex does not exist!.";
-					try {
-						Tuple2<K, Message> next = messageIter.next();
-						message = "Target vertex '" + next.f0 + "' does not exist!.";
-					} catch (Throwable t) {}
-					throw new Exception(message);
-				} else {
-					throw new Exception();
-				}
-			}
-		}
-	}
-
-	/*
-	 * UDF that encapsulates the message sending function for graphs where the edges have an associated value.
-	 */
-	private static abstract class MessagingUdfWithEdgeValues<K, VVWithDegrees, VV, Message, EV>
-		extends RichCoGroupFunction<Edge<K, EV>, Vertex<K, VVWithDegrees>, Tuple2<K, Message>>
-		implements ResultTypeQueryable<Tuple2<K, Message>>
-	{
-		private static final long serialVersionUID = 1L;
-		
-		final MessagingFunction<K, VV, Message, EV> messagingFunction;
-		
-		private transient TypeInformation<Tuple2<K, Message>> resultType;
-	
-	
-		private MessagingUdfWithEdgeValues(MessagingFunction<K, VV, Message, EV> messagingFunction,
-				TypeInformation<Tuple2<K, Message>> resultType)
-		{
-			this.messagingFunction = messagingFunction;
-			this.resultType = resultType;
-		}
-		
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
-				this.messagingFunction.init(getIterationRuntimeContext());
-			}
-			
-			this.messagingFunction.preSuperstep();
-		}
-		
-		@Override
-		public void close() throws Exception {
-			this.messagingFunction.postSuperstep();
-		}
-		
-		@Override
-		public TypeInformation<Tuple2<K, Message>> getProducedType() {
-			return this.resultType;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class MessagingUdfWithEVsSimpleVV<K, VV, Message, EV>
-		extends MessagingUdfWithEdgeValues<K, VV, VV, Message, EV> {
-
-		private MessagingUdfWithEVsSimpleVV(MessagingFunction<K, VV, Message, EV> messagingFunction,
-			TypeInformation<Tuple2<K, Message>> resultType) {
-			super(messagingFunction, resultType);
-		}
-
-		@Override
-		public void coGroup(Iterable<Edge<K, EV>> edges,
-							Iterable<Vertex<K, VV>> state,
-							Collector<Tuple2<K, Message>> out) throws Exception {
-			final Iterator<Vertex<K, VV>> stateIter = state.iterator();
-		
-			if (stateIter.hasNext()) {
-				Vertex<K, VV> newVertexState = stateIter.next();
-				messagingFunction.set((Iterator<?>) edges.iterator(), out);
-				messagingFunction.sendMessages(newVertexState);
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class MessagingUdfWithEVsVVWithDegrees<K, VV, Message, EV>
-		extends MessagingUdfWithEdgeValues<K, Tuple3<VV, Long, Long>, VV, Message, EV> {
-
-		private Vertex<K, VV> nextVertex = new Vertex<K, VV>();
-
-		private MessagingUdfWithEVsVVWithDegrees(MessagingFunction<K, VV, Message, EV> messagingFunction,
-				TypeInformation<Tuple2<K, Message>> resultType) {
-			super(messagingFunction, resultType);
-		}
-
-		@Override
-		public void coGroup(Iterable<Edge<K, EV>> edges, Iterable<Vertex<K, Tuple3<VV, Long, Long>>> state,
-				Collector<Tuple2<K, Message>> out) throws Exception {
-
-			final Iterator<Vertex<K, Tuple3<VV, Long, Long>>> stateIter = state.iterator();
-		
-			if (stateIter.hasNext()) {
-				Vertex<K, Tuple3<VV, Long, Long>> vertexWithDegrees = stateIter.next();
-
-				nextVertex.setField(vertexWithDegrees.f0, 0);
-				nextVertex.setField(vertexWithDegrees.f1.f0, 1);
-
-				messagingFunction.setInDegree(vertexWithDegrees.f1.f1);
-				messagingFunction.setOutDegree(vertexWithDegrees.f1.f2);
-
-				messagingFunction.set((Iterator<?>) edges.iterator(), out);
-				messagingFunction.sendMessages(nextVertex);
-			}
-		}
-	}
-
-
-	// --------------------------------------------------------------------------------------------
-	//  UTIL methods
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Method that builds the messaging function using a coGroup operator for a simple vertex(without
-	 * degrees).
-	 * It afterwards configures the function with a custom name and broadcast variables.
-	 *
-	 * @param iteration
-	 * @param messageTypeInfo
-	 * @param whereArg the argument for the where within the coGroup
-	 * @param equalToArg the argument for the equalTo within the coGroup
-	 * @return the messaging function
-	 */
-	private CoGroupOperator<?, ?, Tuple2<K, Message>> buildMessagingFunction(
-			DeltaIteration<Vertex<K, VV>, Vertex<K, VV>> iteration,
-			TypeInformation<Tuple2<K, Message>> messageTypeInfo, int whereArg, int equalToArg) {
-
-		// build the messaging function (co group)
-		CoGroupOperator<?, ?, Tuple2<K, Message>> messages;
-		MessagingUdfWithEdgeValues<K, VV, VV, Message, EV> messenger =
-				new MessagingUdfWithEVsSimpleVV<K, VV, Message, EV>(messagingFunction, messageTypeInfo);
-
-		messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(whereArg)
-				.equalTo(equalToArg).with(messenger);
-
-		// configure coGroup message function with name and broadcast variables
-		messages = messages.name("Messaging");
-		if(this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getMessagingBcastVars()) {
-				messages = messages.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-
-		return messages;
-	}
-
-	/**
-	 * Method that builds the messaging function using a coGroup operator for a vertex
-	 * containing degree information.
-	 * It afterwards configures the function with a custom name and broadcast variables.
-	 *
-	 * @param iteration
-	 * @param messageTypeInfo
-	 * @param whereArg the argument for the where within the coGroup
-	 * @param equalToArg the argument for the equalTo within the coGroup
-	 * @return the messaging function
-	 */
-	private CoGroupOperator<?, ?, Tuple2<K, Message>> buildMessagingFunctionVerticesWithDegrees(
-			DeltaIteration<Vertex<K, Tuple3<VV, Long, Long>>, Vertex<K, Tuple3<VV, Long, Long>>> iteration,
-			TypeInformation<Tuple2<K, Message>> messageTypeInfo, int whereArg, int equalToArg) {
-
-		// build the messaging function (co group)
-		CoGroupOperator<?, ?, Tuple2<K, Message>> messages;
-		MessagingUdfWithEdgeValues<K, Tuple3<VV, Long, Long>, VV, Message, EV> messenger =
-				new MessagingUdfWithEVsVVWithDegrees<K, VV, Message, EV>(messagingFunction, messageTypeInfo);
-
-		messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(whereArg)
-				.equalTo(equalToArg).with(messenger);
-
-		// configure coGroup message function with name and broadcast variables
-		messages = messages.name("Messaging");
-
-		if (this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getMessagingBcastVars()) {
-				messages = messages.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-
-		return messages;
-	}
-
-	/**
-	 * Helper method which sets up an iteration with the given vertex value(either simple or with degrees)
-	 *
-	 * @param iteration
-	 */
-
-	private void setUpIteration(DeltaIteration<?, ?> iteration) {
-
-		// set up the iteration operator
-		if (this.configuration != null) {
-
-			iteration.name(this.configuration.getName("Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")"));
-			iteration.parallelism(this.configuration.getParallelism());
-			iteration.setSolutionSetUnManaged(this.configuration.isSolutionSetUnmanagedMemory());
-
-			// register all aggregators
-			for (Map.Entry<String, Aggregator<?>> entry : this.configuration.getAggregators().entrySet()) {
-				iteration.registerAggregator(entry.getKey(), entry.getValue());
-			}
-		}
-		else {
-			// no configuration provided; set default name
-			iteration.name("Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")");
-		}
-	}
-
-	/**
-	 * Creates the operator that represents this vertex centric graph computation for a simple vertex.
-	 *
-	 * @param messagingDirection
-	 * @param messageTypeInfo
-	 * @return the operator
-	 */
-	private DataSet<Vertex<K, VV>> createResultSimpleVertex(EdgeDirection messagingDirection,
-		TypeInformation<Tuple2<K, Message>> messageTypeInfo) {
-
-		DataSet<Tuple2<K, Message>> messages;
-
-		TypeInformation<Vertex<K, VV>> vertexTypes = initialVertices.getType();
-
-		final DeltaIteration<Vertex<K, VV>,	Vertex<K, VV>> iteration =
-				initialVertices.iterateDelta(initialVertices, this.maximumNumberOfIterations, 0);
-				setUpIteration(iteration);
-
-		switch (messagingDirection) {
-			case IN:
-				messages = buildMessagingFunction(iteration, messageTypeInfo, 1, 0);
-				break;
-			case OUT:
-				messages = buildMessagingFunction(iteration, messageTypeInfo, 0, 0);
-				break;
-			case ALL:
-				messages = buildMessagingFunction(iteration, messageTypeInfo, 1, 0)
-						.union(buildMessagingFunction(iteration, messageTypeInfo, 0, 0)) ;
-				break;
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-
-		VertexUpdateUdf<K, VV, Message> updateUdf =
-				new VertexUpdateUdfSimpleVV<K, VV, Message>(updateFunction, vertexTypes);
-
-		// build the update function (co group)
-		CoGroupOperator<?, ?, Vertex<K, VV>> updates =
-				messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf);
-
-		configureUpdateFunction(updates);
-
-		return iteration.closeWith(updates, updates);
-	}
-
-	/**
-	 * Creates the operator that represents this vertex centric graph computation for a vertex with in
-	 * and out degrees added to the vertex value.
-	 *
-	 * @param graph
-	 * @param messagingDirection
-	 * @param messageTypeInfo
-	 * @return the operator
-	 */
-	@SuppressWarnings("serial")
-	private DataSet<Vertex<K, VV>> createResultVerticesWithDegrees(Graph<K, VV, EV> graph, EdgeDirection messagingDirection,
-			TypeInformation<Tuple2<K, Message>> messageTypeInfo) {
-
-		DataSet<Tuple2<K, Message>> messages;
-
-		this.updateFunction.setOptDegrees(this.configuration.isOptDegrees());
-
-		DataSet<Tuple2<K, Long>> inDegrees = graph.inDegrees();
-		DataSet<Tuple2<K, Long>> outDegrees = graph.outDegrees();
-
-		DataSet<Tuple3<K, Long, Long>> degrees = inDegrees.join(outDegrees).where(0).equalTo(0)
-				.with(new FlatJoinFunction<Tuple2<K, Long>, Tuple2<K, Long>, Tuple3<K, Long, Long>>() {
-
-					@Override
-					public void join(Tuple2<K, Long> first, Tuple2<K, Long> second,	Collector<Tuple3<K, Long, Long>> out) {
-						out.collect(new Tuple3<K, Long, Long>(first.f0, first.f1, second.f1));
-					}
-				}).withForwardedFieldsFirst("f0;f1").withForwardedFieldsSecond("f1");
-
-		DataSet<Vertex<K, Tuple3<VV, Long, Long>>> verticesWithDegrees = initialVertices
-				.join(degrees).where(0).equalTo(0)
-				.with(new FlatJoinFunction<Vertex<K,VV>, Tuple3<K,Long,Long>, Vertex<K, Tuple3<VV, Long, Long>>>() {
-					@Override
-					public void join(Vertex<K, VV> vertex, Tuple3<K, Long, Long> degrees,
-									Collector<Vertex<K, Tuple3<VV, Long, Long>>> out) throws Exception {
-
-						out.collect(new Vertex<K, Tuple3<VV, Long, Long>>(vertex.getId(),
-								new Tuple3<VV, Long, Long>(vertex.getValue(), degrees.f1, degrees.f2)));
-					}
-				}).withForwardedFieldsFirst("f0");
-
-		// add type info
-		TypeInformation<Vertex<K, Tuple3<VV, Long, Long>>> vertexTypes = verticesWithDegrees.getType();
-
-		final DeltaIteration<Vertex<K, Tuple3<VV, Long, Long>>,	Vertex<K, Tuple3<VV, Long, Long>>> iteration =
-				verticesWithDegrees.iterateDelta(verticesWithDegrees, this.maximumNumberOfIterations, 0);
-				setUpIteration(iteration);
-
-		switch (messagingDirection) {
-			case IN:
-				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 1, 0);
-				break;
-			case OUT:
-				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 0, 0);
-				break;
-			case ALL:
-				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 1, 0)
-						.union(buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 0, 0)) ;
-				break;
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		VertexUpdateUdf<K, Tuple3<VV, Long, Long>, Message> updateUdf =
-				new VertexUpdateUdfVVWithDegrees(updateFunction, vertexTypes);
-
-		// build the update function (co group)
-		CoGroupOperator<?, ?, Vertex<K, Tuple3<VV, Long, Long>>> updates =
-				messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf);
-
-		configureUpdateFunction(updates);
-
-		return iteration.closeWith(updates, updates).map(
-				new MapFunction<Vertex<K, Tuple3<VV, Long, Long>>, Vertex<K, VV>>() {
-
-					public Vertex<K, VV> map(Vertex<K, Tuple3<VV, Long, Long>> vertex) {
-						return new Vertex<K, VV>(vertex.getId(), vertex.getValue().f0);
-					}
-				});
-	}
-
-	private <VVWithDegree> void configureUpdateFunction(CoGroupOperator<?, ?, Vertex<K, VVWithDegree>> updates) {
-
-		// configure coGroup update function with name and broadcast variables
-		updates = updates.name("Vertex State Updates");
-		if (this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getUpdateBcastVars()) {
-				updates = updates.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-
-		// let the operator know that we preserve the key field
-		updates.withForwardedFieldsFirst("0").withForwardedFieldsSecond("0");
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
deleted file mode 100644
index 248925b..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
+++ /dev/null
@@ -1,253 +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.spargel;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.Collector;
-
-/**
- * This class must be extended by functions that compute the state of the vertex depending on the old state and the
- * incoming messages. The central method is {@link #updateVertex(Comparable, Object, MessageIterator)}, which is
- * invoked once per vertex per superstep.
- * 
- * <K> The vertex key type.
- * <VV> The vertex value type.
- * <Message> The message type.
- */
-public abstract class VertexUpdateFunction<K, VV, Message> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	// --------------------------------------------------------------------------------------------
-	//  Attributes that allow vertices to access their in/out degrees and the total number of vertices
-	//  inside an iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private long numberOfVertices = -1L;
-
-	/**
-	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
-	 * option has been set; -1 otherwise.
-	 */
-	public long getNumberOfVertices() {
-		return numberOfVertices;
-	}
-
-	void setNumberOfVertices(long numberOfVertices) {
-		this.numberOfVertices = numberOfVertices;
-	}
-
-	//---------------------------------------------------------------------------------------------
-
-	private boolean optDegrees;
-
-	boolean isOptDegrees() {
-		return optDegrees;
-	}
-
-	void setOptDegrees(boolean optDegrees) {
-		this.optDegrees = optDegrees;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Public API Methods
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * This method is invoked once per vertex per superstep. It receives the current state of the vertex, as well as
-	 * the incoming messages. It may set a new vertex state via {@link #setNewVertexValue(Object)}. If the vertex
-	 * state is changed, it will trigger the sending of messages via the {@link MessagingFunction}.
-	 * 
-	 * @param vertex The vertex.
-	 * @param inMessages The incoming messages to this vertex.
-	 * 
-	 * @throws Exception The computation may throw exceptions, which causes the superstep to fail.
-	 */
-	public abstract void updateVertex(Vertex<K, VV> vertex, MessageIterator<Message> inMessages) throws Exception;
-	
-	/**
-	 * This method is executed one per superstep before the vertex update function is invoked for each vertex.
-	 * 
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
-	 */
-	public void preSuperstep() throws Exception {}
-	
-	/**
-	 * This method is executed one per superstep after the vertex update function has been invoked for each vertex.
-	 * 
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
-	 */
-	public void postSuperstep() throws Exception {}
-	
-	/**
-	 * Sets the new value of this vertex. Setting a new value triggers the sending of outgoing messages from this vertex.
-	 *
-	 * This should be called at most once per updateVertex.
-	 * 
-	 * @param newValue The new vertex value.
-	 */
-	public void setNewVertexValue(VV newValue) {
-		if(setNewVertexValueCalled) {
-			throw new IllegalStateException("setNewVertexValue should only be called at most once per updateVertex");
-		}
-		setNewVertexValueCalled = true;
-		if(isOptDegrees()) {
-			outValWithDegrees.f1.f0 = newValue;
-			outWithDegrees.collect(outValWithDegrees);
-		} else {
-			outVal.setValue(newValue);
-			out.collect(outVal);
-		}
-	}
-	
-	/**
-	 * Gets the number of the superstep, starting at <tt>1</tt>.
-	 * 
-	 * @return The number of the current superstep.
-	 */
-	public int getSuperstepNumber() {
-		return this.runtimeContext.getSuperstepNumber();
-	}
-	
-	/**
-	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
-	 * all aggregates globally once per superstep and makes them available in the next superstep.
-	 * 
-	 * @param name The name of the aggregator.
-	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
-	 */
-	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-		return this.runtimeContext.<T>getIterationAggregator(name);
-	}
-	
-	/**
-	 * Get the aggregated value that an aggregator computed in the previous iteration.
-	 * 
-	 * @param name The name of the aggregator.
-	 * @return The aggregated value of the previous iteration.
-	 */
-	public <T extends Value> T getPreviousIterationAggregate(String name) {
-		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
-	}
-	
-	/**
-	 * Gets the broadcast data set registered under the given name. Broadcast data sets
-	 * are available on all parallel instances of a function. They can be registered via
-	 * {@link org.apache.flink.graph.spargel.VertexCentricConfiguration#addBroadcastSetForUpdateFunction(String, org.apache.flink.api.java.DataSet)}.
-	 * 
-	 * @param name The name under which the broadcast set is registered.
-	 * @return The broadcast data set.
-	 */
-	public <T> Collection<T> getBroadcastSet(String name) {
-		return this.runtimeContext.<T>getBroadcastVariable(name);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  internal methods
-	// --------------------------------------------------------------------------------------------
-	
-	private IterationRuntimeContext runtimeContext;
-
-	private Collector<Vertex<K, VV>> out;
-	
-	private Collector<Vertex<K, Tuple3<VV, Long, Long>>> outWithDegrees;
-
-	private Vertex<K, VV> outVal;
-
-	private Vertex<K, Tuple3<VV, Long, Long>> outValWithDegrees;
-
-	private long inDegree = -1;
-
-	private long outDegree = -1;
-
-	private boolean setNewVertexValueCalled;
-
-	void init(IterationRuntimeContext context) {
-		this.runtimeContext = context;
-	}
-
-	void setOutput(Vertex<K, VV> outVal, Collector<Vertex<K, VV>> out) {
-		this.outVal = outVal;
-		this.out = out;
-		setNewVertexValueCalled = false;
-	}
-
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	<ValueWithDegree> void setOutputWithDegrees(Vertex<K, ValueWithDegree> outVal,
-			Collector out) {
-		this.outValWithDegrees = (Vertex<K, Tuple3<VV, Long, Long>>) outVal;
-		this.outWithDegrees = out;
-		setNewVertexValueCalled = false;
-	}
-
-	/**
-	 * Retrieves the vertex in-degree (number of in-coming edges).
-	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
-	 */
-	public long getInDegree() {
-		return inDegree;
-	}
-
-	void setInDegree(long inDegree) {
-		this.inDegree = inDegree;
-	}
-
-	/**
-	 * Retrieve the vertex out-degree (number of out-going edges).
-	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
-	 */
-	public long getOutDegree() {
-		return outDegree;
-	}
-
-	void setOutDegree(long outDegree) {
-		this.outDegree = outDegree;
-	}
-
-	/**
-	 * In order to hide the Tuple3(actualValue, inDegree, OutDegree) vertex value from the user,
-	 * another function will be called from {@link org.apache.flink.graph.spargel.VertexCentricIteration}.
-	 *
-	 * This function will retrieve the vertex from the vertexState and will set its degrees, afterwards calling
-	 * the regular updateVertex function.
-	 *
-	 * @param vertexState
-	 * @param inMessages
-	 * @throws Exception
-	 */
-	@SuppressWarnings("unchecked")
-	<VertexWithDegree> void updateVertexFromVertexCentricIteration(Vertex<K, VertexWithDegree> vertexState,
-												MessageIterator<Message> inMessages) throws Exception {
-
-		Vertex<K, VV> vertex = new Vertex<K, VV>(vertexState.f0,
-				((Tuple3<VV, Long, Long>)vertexState.getValue()).f0);
-
-		updateVertex(vertex, inMessages);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
deleted file mode 100644
index 0e085b4..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
+++ /dev/null
@@ -1,35 +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.api.java.functions.FunctionAnnotation.ForwardedFields;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-
-@ForwardedFields("f0; f1; f2")
-public class EdgeToTuple3Map<K, EV> implements MapFunction<Edge<K, EV>, Tuple3<K, K, EV>> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Tuple3<K, K, EV> map(Edge<K, EV> edge) {
-		return new Tuple3<K, K, EV>(edge.f0, edge.f1, edge.f2);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
deleted file mode 100644
index 2bd4719..0000000
--- a/flink-staging/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();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
deleted file mode 100644
index e51362b..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
+++ /dev/null
@@ -1,35 +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.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-
-@ForwardedFields("f0; f1")
-public class Tuple2ToVertexMap<K, VV> implements MapFunction<Tuple2<K, VV>, Vertex<K, VV>> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Vertex<K, VV> map(Tuple2<K, VV> vertex) {
-		return new Vertex<K, VV>(vertex.f0, vertex.f1);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java
deleted file mode 100644
index 0db9a51..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.graph.utils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-
-/**
- * create an Edge DataSetfrom a Tuple3 dataset
- *
- * @param <K>
- * @param <EV>
- */
-@ForwardedFields("f0; f1; f2")
-public class Tuple3ToEdgeMap<K, EV> implements MapFunction<Tuple3<K, K, EV>, Edge<K, EV>> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Edge<K, EV> map(Tuple3<K, K, EV> tuple) {
-		return new Edge<K, EV>(tuple.f0, tuple.f1, tuple.f2);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
deleted file mode 100644
index 04d1f47..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
+++ /dev/null
@@ -1,35 +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.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-
-@ForwardedFields("f0; f1")
-public class VertexToTuple2Map<K, VV> implements MapFunction<Vertex<K, VV>, Tuple2<K, VV>> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Tuple2<K, VV> map(Vertex<K, VV> vertex) {
-		return new Tuple2<K, VV>(vertex.f0, vertex.f1);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
deleted file mode 100644
index 75b672c..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
+++ /dev/null
@@ -1,37 +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.validation;
-
-import java.io.Serializable;
-
-import org.apache.flink.graph.Graph;
-
-/**
- * A utility for defining validation criteria for different types of Graphs.
- * 
- * @param <K> the vertex key type
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- */
-@SuppressWarnings("serial")
-public abstract class GraphValidator<K, VV, EV>	implements Serializable {
-
-	public abstract boolean validate(Graph<K, VV, EV> graph) throws Exception;
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
deleted file mode 100644
index 33d469b..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
+++ /dev/null
@@ -1,74 +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.validation;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("serial")
-public class InvalidVertexIdsValidator<K, VV, EV> extends GraphValidator<K, VV, EV> {
-
-	/**
-	 * Checks that the edge set input contains valid vertex Ids, i.e. that they
-	 * also exist in the vertex input set.
-	 * 
-	 * @return a boolean stating whether a graph is valid
-	 *         with respect to its vertex ids.
-	 */
-	@Override
-	public boolean validate(Graph<K, VV, EV> graph) throws Exception {
-		DataSet<Tuple1<K>> edgeIds = graph.getEdges()
-				.flatMap(new MapEdgeIds<K, EV>()).distinct();
-		DataSet<K> invalidIds = graph.getVertices().coGroup(edgeIds).where(0)
-				.equalTo(0).with(new GroupInvalidIds<K, VV>()).first(1);
-
-		return invalidIds.map(new KToTupleMap<K>()).count() == 0;
-	}
-
-	private static final class MapEdgeIds<K, EV> implements FlatMapFunction<Edge<K, EV>, Tuple1<K>> {
-		public void flatMap(Edge<K, EV> edge, Collector<Tuple1<K>> out) {
-			out.collect(new Tuple1<K>(edge.f0));
-			out.collect(new Tuple1<K>(edge.f1));
-		}
-	}
-
-	private static final class GroupInvalidIds<K, VV> implements CoGroupFunction<Vertex<K, VV>, Tuple1<K>, K> {
-		public void coGroup(Iterable<Vertex<K, VV>> vertexId,
-				Iterable<Tuple1<K>> edgeId, Collector<K> out) {
-			if (!(vertexId.iterator().hasNext())) {
-				// found an id that doesn't exist in the vertex set
-				out.collect(edgeId.iterator().next().f0);
-			}
-		}
-	}
-
-	private static final class KToTupleMap<K> implements MapFunction<K, Tuple1<K>> {
-		public Tuple1<K> map(K key) throws Exception {
-			return new Tuple1<K>(key);
-		}
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
deleted file mode 100644
index 2ad203f..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
+++ /dev/null
@@ -1,146 +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.gsa;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.util.FieldList;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
-import org.apache.flink.optimizer.dataproperties.PartitioningProperty;
-import org.apache.flink.optimizer.plan.DualInputPlanNode;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plan.PlanNode;
-import org.apache.flink.optimizer.plan.SingleInputPlanNode;
-import org.apache.flink.optimizer.plan.SinkPlanNode;
-import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
-import org.apache.flink.optimizer.util.CompilerTestBase;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-
-public class GSACompilerTest extends CompilerTestBase {
-
-	private static final long serialVersionUID = 1L;
-
-	@Test
-	public void testGSACompiler() {
-		try {
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(DEFAULT_PARALLELISM);
-			// compose test program
-			{
-				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<Long, Long, NullValue>(
-						1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
-
-				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
-
-				DataSet<Vertex<Long, Long>> result = graph.runGatherSumApplyIteration(
-						new GatherNeighborIds(), new SelectMinId(),
-						new UpdateComponentId(), 100).getVertices();
-				
-				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
-			}
-			
-			Plan p = env.createProgramPlan("GSA Connected Components");
-			OptimizedPlan op = compileNoStats(p);
-			
-			// check the sink
-			SinkPlanNode sink = op.getDataSinks().iterator().next();
-			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
-			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
-			assertEquals(PartitioningProperty.HASH_PARTITIONED, sink.getGlobalProperties().getPartitioning());
-			
-			// check the iteration
-			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
-			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
-			
-			// check the solution set join and the delta
-			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
-			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update function preserves the partitioning
-			
-			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
-			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
-			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
-			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
-			
-			// check the workset set join
-			SingleInputPlanNode sumReducer = (SingleInputPlanNode) ssJoin.getInput1().getSource();
-			SingleInputPlanNode gatherMapper = (SingleInputPlanNode) sumReducer.getInput().getSource();
-			DualInputPlanNode edgeJoin = (DualInputPlanNode) gatherMapper.getInput().getSource(); 
-			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
-			// input1 is the workset
-			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput1().getShipStrategy());
-			// input2 is the edges
-			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput2().getShipStrategy());
-			assertTrue(edgeJoin.getInput2().getTempMode().isCached());
-
-			assertEquals(new FieldList(0), edgeJoin.getInput2().getShipStrategyKeys());
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitVertices	implements MapFunction<Long, Long> {
-
-		public Long map(Long vertexId) {
-			return vertexId;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
-
-		public Long gather(Neighbor<Long, NullValue> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
-
-		public Long sum(Long newValue, Long currentValue) {
-			return Math.min(newValue, currentValue);
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class UpdateComponentId extends ApplyFunction<Long, Long, Long> {
-
-		public void apply(Long summedValue, Long origValue) {
-			if (summedValue < origValue) {
-				setResult(summedValue);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
deleted file mode 100644
index ced7508..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
+++ /dev/null
@@ -1,164 +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.gsa;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-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.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.operators.DeltaIterationResultSet;
-import org.apache.flink.api.java.operators.SingleInputUdfOperator;
-import org.apache.flink.api.java.operators.TwoInputUdfOperator;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-
-public class GSATranslationTest {
-
-	@Test
-	public void testTranslation() {
-		try {
-			final String ITERATION_NAME = "Test Name";
-			
-			final String AGGREGATOR_NAME = "AggregatorName";
-			
-			final String BC_SET_GATHER_NAME = "gather messages";
-			
-			final String BC_SET_SUM_NAME = "sum updates";
-
-			final String BC_SET_APLLY_NAME = "apply updates";
-
-			final int NUM_ITERATIONS = 13;
-			
-			final int ITERATION_parallelism = 77;
-			
-			
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
-			DataSet<Long> bcGather = env.fromElements(1L);
-			DataSet<Long> bcSum = env.fromElements(1L);
-			DataSet<Long> bcApply = env.fromElements(1L);
-
-			DataSet<Vertex<Long, Long>> result;
-
-			// ------------ construct the test program ------------------
-			{
-
-				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<Long, Long, NullValue>(
-						1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
-
-				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
-
-				GSAConfiguration parameters = new GSAConfiguration();
-
-				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
-				parameters.setName(ITERATION_NAME);
-				parameters.setParallelism(ITERATION_parallelism);
-				parameters.addBroadcastSetForGatherFunction(BC_SET_GATHER_NAME, bcGather);
-				parameters.addBroadcastSetForSumFunction(BC_SET_SUM_NAME, bcSum);
-				parameters.addBroadcastSetForApplyFunction(BC_SET_APLLY_NAME, bcApply);
-
-				result = graph.runGatherSumApplyIteration(
-						new GatherNeighborIds(), new SelectMinId(),
-						new UpdateComponentId(), NUM_ITERATIONS, parameters).getVertices();
-				
-				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
-			}
-			
-			
-			// ------------- validate the java program ----------------
-			
-			assertTrue(result instanceof DeltaIterationResultSet);
-			
-			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
-			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
-			
-			// check the basic iteration properties
-			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
-			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
-			assertEquals(ITERATION_parallelism, iteration.getParallelism());
-			assertEquals(ITERATION_NAME, iteration.getName());
-			
-			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
-			
-			// validate that the semantic properties are set as they should
-			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
-
-			SingleInputUdfOperator<?, ?, ?> sumReduce = (SingleInputUdfOperator<?, ?, ?>) solutionSetJoin.getInput1();
-			SingleInputUdfOperator<?, ?, ?> gatherMap = (SingleInputUdfOperator<?, ?, ?>) sumReduce.getInput();
-
-			// validate that the broadcast sets are forwarded
-			assertEquals(bcGather, gatherMap.getBroadcastSets().get(BC_SET_GATHER_NAME));
-			assertEquals(bcSum, sumReduce.getBroadcastSets().get(BC_SET_SUM_NAME));
-			assertEquals(bcApply, solutionSetJoin.getBroadcastSets().get(BC_SET_APLLY_NAME));
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitVertices	implements MapFunction<Long, Long> {
-
-		public Long map(Long vertexId) {
-			return vertexId;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
-
-		public Long gather(Neighbor<Long, NullValue> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
-
-		public Long sum(Long newValue, Long currentValue) {
-			return Math.min(newValue, currentValue);
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class UpdateComponentId extends ApplyFunction<Long, Long, Long> {
-
-		public void apply(Long summedValue, Long origValue) {
-			if (summedValue < origValue) {
-				setResult(summedValue);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
deleted file mode 100644
index 7a8143a..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
+++ /dev/null
@@ -1,214 +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.spargel;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.util.FieldList;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.util.CompilerTestBase;
-import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.optimizer.plan.DualInputPlanNode;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plan.PlanNode;
-import org.apache.flink.optimizer.plan.SinkPlanNode;
-import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.graph.library.ConnectedComponents;
-import org.apache.flink.graph.utils.Tuple2ToVertexMap;
-
-
-public class SpargelCompilerTest extends CompilerTestBase {
-
-	private static final long serialVersionUID = 1L;
-
-	@SuppressWarnings("serial")
-	@Test
-	public void testSpargelCompiler() {
-		try {
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(DEFAULT_PARALLELISM);
-			// compose test program
-			{
-
-				DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
-						new Tuple2<Long, Long>(1L, 1L), new Tuple2<Long, Long>(2L, 2L))
-						.map(new Tuple2ToVertexMap<Long, Long>());
-
-				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<Long, Long>(1L, 2L))
-					.map(new MapFunction<Tuple2<Long,Long>, Edge<Long, NullValue>>() {
-
-						public Edge<Long, NullValue> map(Tuple2<Long, Long> edge) {
-							return new Edge<Long, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
-						}
-				});
-
-				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
-				
-				DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
-						new ConnectedComponents.CCUpdater<Long>(),
-						new ConnectedComponents.CCMessenger<Long>(), 100)
-						.getVertices();
-				
-				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
-			}
-			
-			Plan p = env.createProgramPlan("Spargel Connected Components");
-			OptimizedPlan op = compileNoStats(p);
-			
-			// check the sink
-			SinkPlanNode sink = op.getDataSinks().iterator().next();
-			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
-			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
-			
-			// check the iteration
-			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
-			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
-			
-			// check the solution set join and the delta
-			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
-			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update functions preserves the partitioning
-			
-			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
-			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
-			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
-			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
-			
-			// check the workset set join
-			DualInputPlanNode edgeJoin = (DualInputPlanNode) ssJoin.getInput1().getSource();
-			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
-			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput1().getShipStrategy());
-			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput2().getShipStrategy());
-			assertTrue(edgeJoin.getInput1().getTempMode().isCached());
-			
-			assertEquals(new FieldList(0), edgeJoin.getInput1().getShipStrategyKeys());
-			
-			// check that the initial partitioning is pushed out of the loop
-			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput1().getShipStrategy());
-			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput2().getShipStrategy());
-			assertEquals(new FieldList(0), iteration.getInput1().getShipStrategyKeys());
-			assertEquals(new FieldList(0), iteration.getInput2().getShipStrategyKeys());
-			
-			// check that the initial workset sort is outside the loop
-			assertEquals(LocalStrategy.SORT, iteration.getInput2().getLocalStrategy());
-			assertEquals(new FieldList(0), iteration.getInput2().getLocalStrategyKeys());
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@SuppressWarnings("serial")
-	@Test
-	public void testSpargelCompilerWithBroadcastVariable() {
-		try {
-			final String BC_VAR_NAME = "borat variable";
-			
-			
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(DEFAULT_PARALLELISM);
-			// compose test program
-			{
-				DataSet<Long> bcVar = env.fromElements(1L);
-
-				DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
-						new Tuple2<Long, Long>(1L, 1L), new Tuple2<Long, Long>(2L, 2L))
-						.map(new Tuple2ToVertexMap<Long, Long>());
-
-				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<Long, Long>(1L, 2L))
-						.map(new MapFunction<Tuple2<Long,Long>, Edge<Long, NullValue>>() {
-
-							public Edge<Long, NullValue> map(Tuple2<Long, Long> edge) {
-								return new Edge<Long, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
-							}
-					});
-
-				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
-
-				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-				parameters.addBroadcastSetForMessagingFunction(BC_VAR_NAME, bcVar);
-				parameters.addBroadcastSetForUpdateFunction(BC_VAR_NAME, bcVar);
-
-				DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
-						new ConnectedComponents.CCUpdater<Long>(),
-						new ConnectedComponents.CCMessenger<Long>(), 100)
-						.getVertices();
-					
-				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
-
-			}
-			
-			Plan p = env.createProgramPlan("Spargel Connected Components");
-			OptimizedPlan op = compileNoStats(p);
-			
-			// check the sink
-			SinkPlanNode sink = op.getDataSinks().iterator().next();
-			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
-			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
-			
-			// check the iteration
-			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
-			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
-			
-			// check the solution set join and the delta
-			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
-			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update functions preserves the partitioning
-			
-			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
-			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
-			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
-			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
-			
-			// check the workset set join
-			DualInputPlanNode edgeJoin = (DualInputPlanNode) ssJoin.getInput1().getSource();
-			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
-			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput1().getShipStrategy());
-			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput2().getShipStrategy());
-			assertTrue(edgeJoin.getInput1().getTempMode().isCached());
-			
-			assertEquals(new FieldList(0), edgeJoin.getInput1().getShipStrategyKeys());
-			
-			// check that the initial partitioning is pushed out of the loop
-			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput1().getShipStrategy());
-			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput2().getShipStrategy());
-			assertEquals(new FieldList(0), iteration.getInput1().getShipStrategyKeys());
-			assertEquals(new FieldList(0), iteration.getInput2().getShipStrategyKeys());
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}


[16/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
new file mode 100644
index 0000000..bb3a131
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
@@ -0,0 +1,231 @@
+/*
+ * 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.spargel;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.junit.Test;
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIterationResultSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.TwoInputUdfOperator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.NullValue;
+
+@SuppressWarnings("serial")
+public class SpargelTranslationTest {
+
+	@Test
+	public void testTranslationPlainEdges() {
+		try {
+			final String ITERATION_NAME = "Test Name";
+			
+			final String AGGREGATOR_NAME = "AggregatorName";
+			
+			final String BC_SET_MESSAGES_NAME = "borat messages";
+			
+			final String BC_SET_UPDATES_NAME = "borat updates";
+			;
+			final int NUM_ITERATIONS = 13;
+			
+			final int ITERATION_parallelism = 77;
+			
+			
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			
+			DataSet<Long> bcMessaging = env.fromElements(1L);
+			DataSet<Long> bcUpdate = env.fromElements(1L);
+			
+			DataSet<Vertex<String, Double>> result;
+			
+			// ------------ construct the test program ------------------
+			{
+				
+				DataSet<Tuple2<String, Double>> initialVertices = env.fromElements(new Tuple2<String, Double>("abc", 3.44));
+
+				DataSet<Tuple2<String, String>> edges = env.fromElements(new Tuple2<String, String>("a", "c"));
+
+				Graph<String, Double, NullValue> graph = Graph.fromTupleDataSet(initialVertices,
+						edges.map(new MapFunction<Tuple2<String,String>, Tuple3<String, String, NullValue>>() {
+
+							public Tuple3<String, String, NullValue> map(
+									Tuple2<String, String> edge) {
+								return new Tuple3<String, String, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
+							}
+						}), env);
+
+				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+				parameters.addBroadcastSetForMessagingFunction(BC_SET_MESSAGES_NAME, bcMessaging);
+				parameters.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcUpdate);
+				parameters.setName(ITERATION_NAME);
+				parameters.setParallelism(ITERATION_parallelism);
+				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
+
+				result = graph.runVertexCentricIteration(new UpdateFunction(), new MessageFunctionNoEdgeValue(),
+						NUM_ITERATIONS, parameters).getVertices();
+
+				result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
+			}
+			
+			
+			// ------------- validate the java program ----------------
+			
+			assertTrue(result instanceof DeltaIterationResultSet);
+			
+			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
+			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
+			
+			// check the basic iteration properties
+			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
+			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
+			assertEquals(ITERATION_NAME, iteration.getName());
+			
+			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
+			
+			// validate that the semantic properties are set as they should
+			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
+			
+			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
+			
+			// validate that the broadcast sets are forwarded
+			assertEquals(bcUpdate, solutionSetJoin.getBroadcastSets().get(BC_SET_UPDATES_NAME));
+			assertEquals(bcMessaging, edgesJoin.getBroadcastSets().get(BC_SET_MESSAGES_NAME));
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTranslationPlainEdgesWithForkedBroadcastVariable() {
+		try {
+			final String ITERATION_NAME = "Test Name";
+			
+			final String AGGREGATOR_NAME = "AggregatorName";
+			
+			final String BC_SET_MESSAGES_NAME = "borat messages";
+			
+			final String BC_SET_UPDATES_NAME = "borat updates";
+			;
+			final int NUM_ITERATIONS = 13;
+			
+			final int ITERATION_parallelism = 77;
+			
+			
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			
+			DataSet<Long> bcVar = env.fromElements(1L);
+			
+			DataSet<Vertex<String, Double>> result;
+			
+			// ------------ construct the test program ------------------
+			{
+
+				DataSet<Tuple2<String, Double>> initialVertices = env.fromElements(new Tuple2<String, Double>("abc", 3.44));
+
+				DataSet<Tuple2<String, String>> edges = env.fromElements(new Tuple2<String, String>("a", "c"));
+
+				Graph<String, Double, NullValue> graph = Graph.fromTupleDataSet(initialVertices,
+						edges.map(new MapFunction<Tuple2<String,String>, Tuple3<String, String, NullValue>>() {
+
+							public Tuple3<String, String, NullValue> map(
+									Tuple2<String, String> edge) {
+								return new Tuple3<String, String, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
+							}
+						}), env);
+
+				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+				parameters.addBroadcastSetForMessagingFunction(BC_SET_MESSAGES_NAME, bcVar);
+				parameters.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcVar);
+				parameters.setName(ITERATION_NAME);
+				parameters.setParallelism(ITERATION_parallelism);
+				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
+				
+				result = graph.runVertexCentricIteration(new UpdateFunction(), new MessageFunctionNoEdgeValue(),
+						NUM_ITERATIONS, parameters).getVertices();
+
+				result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
+			}
+			
+			
+			// ------------- validate the java program ----------------
+			
+			assertTrue(result instanceof DeltaIterationResultSet);
+			
+			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
+			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
+			
+			// check the basic iteration properties
+			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
+			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
+			assertEquals(ITERATION_NAME, iteration.getName());
+			
+			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
+			
+			// validate that the semantic properties are set as they should
+			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
+			
+			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
+			
+			// validate that the broadcast sets are forwarded
+			assertEquals(bcVar, solutionSetJoin.getBroadcastSets().get(BC_SET_UPDATES_NAME));
+			assertEquals(bcVar, edgesJoin.getBroadcastSets().get(BC_SET_MESSAGES_NAME));
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static class UpdateFunction extends VertexUpdateFunction<String, Double, Long> {
+
+		@Override
+		public void updateVertex(Vertex<String, Double> vertex, MessageIterator<Long> inMessages) {}
+	}
+	
+	public static class MessageFunctionNoEdgeValue extends MessagingFunction<String, Double, Long, NullValue> {
+
+		@Override
+		public void sendMessages(Vertex<String, Double> vertex) {}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
new file mode 100644
index 0000000..3fbd0bc
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph.test;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.graph.utils.VertexToTuple2Map;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class CollectionModeSuperstepITCase {
+
+	/**
+	 * Dummy iteration to test that the supersteps are correctly incremented
+	 * and can be retrieved from inside the updated and messaging functions.
+	 * All vertices start with value 1 and increase their value by 1
+	 * in each iteration. 
+	 */
+	@Test
+	public void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
+				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
+		
+		Graph<Long, Long, Long> result = graph.runVertexCentricIteration(
+				new UpdateFunction(), new MessageFunction(), 10);
+
+		result.getVertices().map(
+				new VertexToTuple2Map<Long, Long>()).output(
+						new DiscardingOutputFormat<Tuple2<Long, Long>>());
+		env.execute();
+	}
+	
+	public static final class UpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			long superstep = getSuperstepNumber();
+			Assert.assertEquals(true, vertex.getValue() == superstep);
+			setNewVertexValue(vertex.getValue() + 1);
+		}
+	}
+	
+	public static final class MessageFunction extends MessagingFunction<Long, Long, Long, Long> {
+		@Override
+		public void sendMessages(Vertex<Long, Long> vertex) {
+			long superstep = getSuperstepNumber();
+			Assert.assertEquals(true, vertex.getValue() == superstep);
+			//send message to keep vertices active
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+
+	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
+
+		public Long map(Vertex<Long, Long> value) {
+			return 1l;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
new file mode 100644
index 0000000..67d32a8
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
@@ -0,0 +1,404 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+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.Tuple1;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.gsa.ApplyFunction;
+import org.apache.flink.graph.gsa.GSAConfiguration;
+import org.apache.flink.graph.gsa.GatherFunction;
+import org.apache.flink.graph.gsa.GatherSumApplyIteration;
+import org.apache.flink.graph.gsa.Neighbor;
+import org.apache.flink.graph.gsa.SumFunction;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.LongValue;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.HashSet;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class GatherSumApplyConfigurationITCase extends MultipleProgramsTestBase {
+
+	public GatherSumApplyConfigurationITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private String expectedResult;
+
+	@Test
+	public void testRunWithConfiguration() throws Exception {
+		/*
+		 * Test Graph's runGatherSumApplyIteration when configuration parameters are provided
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
+
+		// create the configuration object
+		GSAConfiguration parameters = new GSAConfiguration();
+
+		parameters.addBroadcastSetForGatherFunction("gatherBcastSet", env.fromElements(1, 2, 3));
+		parameters.addBroadcastSetForSumFunction("sumBcastSet", env.fromElements(4, 5, 6));
+		parameters.addBroadcastSetForApplyFunction("applyBcastSet", env.fromElements(7, 8, 9));
+		parameters.registerAggregator("superstepAggregator", new LongSumAggregator());
+		parameters.setOptNumVertices(true);
+
+		Graph<Long, Long, Long> res = graph.runGatherSumApplyIteration(new Gather(), new Sum(),
+				new Apply(), 10, parameters);
+
+        DataSet<Vertex<Long, Long>> data = res.getVertices();
+        List<Vertex<Long, Long>> result= data.collect();
+
+		expectedResult = "1,11\n" +
+				"2,11\n" +
+				"3,11\n" +
+				"4,11\n" +
+				"5,11";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationConfiguration() throws Exception {
+
+		/*
+		 * Test name, parallelism and solutionSetUnmanaged parameters
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		GatherSumApplyIteration<Long, Long, Long, Long> iteration = GatherSumApplyIteration
+				.withEdges(TestGraphUtils.getLongLongEdgeData(env), new DummyGather(),
+						new DummySum(), new DummyApply(), 10);
+
+		GSAConfiguration parameters = new GSAConfiguration();
+		parameters.setName("gelly iteration");
+		parameters.setParallelism(2);
+		parameters.setSolutionSetUnmanagedMemory(true);
+
+		iteration.configure(parameters);
+
+		Assert.assertEquals("gelly iteration", iteration.getIterationConfiguration().getName(""));
+		Assert.assertEquals(2, iteration.getIterationConfiguration().getParallelism());
+		Assert.assertEquals(true, iteration.getIterationConfiguration().isSolutionSetUnmanagedMemory());
+
+		DataSet<Vertex<Long, Long>> data = TestGraphUtils.getLongLongVertexData(env).runOperation(iteration);
+        List<Vertex<Long, Long>> result= data.collect();
+        
+		expectedResult = "1,11\n" +
+				"2,12\n" +
+				"3,13\n" +
+				"4,14\n" +
+				"5,15";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationDefaultDirection() throws Exception {
+
+		/*
+		 * Test that if no direction parameter is given, the iteration works as before
+		 * (i.e. it gathers information from the IN edges and neighbors and the information is calculated for an OUT edge
+		 * Default direction parameter is OUT for the GatherSumApplyIterations)
+		 * When data is gathered from the IN edges the Gather Sum and Apply functions
+		 * set the set of vertices which have path to a vertex as the value of that vertex
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
+
+		edges.remove(0);
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
+				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
+				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4)
+				.getVertices();
+
+		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
+
+		expectedResult = "1,[1, 2, 3, 4, 5]\n"
+						+"2,[2]\n"
+						+"3,[1, 2, 3, 4, 5]\n"
+						+"4,[1, 2, 3, 4, 5]\n"
+						+"5,[1, 2, 3, 4, 5]\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationDirectionIN() throws Exception {
+
+		/*
+		 * Test that if the direction parameter IN is given, the iteration works as expected
+		 * (i.e. it gathers information from the OUT edges and neighbors and the information is calculated for an IN edge
+		 * When data is gathered from the OUT edges the Gather Sum and Apply functions
+		 * set the set of vertices which have path from a vertex as the value of that vertex
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		GSAConfiguration parameters = new GSAConfiguration();
+
+		parameters.setDirection(EdgeDirection.IN);
+
+		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
+
+		edges.remove(0);
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
+				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
+				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4,
+																								parameters)
+				.getVertices();
+		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
+
+		expectedResult = "1,[1, 3, 4, 5]\n"
+				+"2,[1, 2, 3, 4, 5]\n"
+				+"3,[1, 3, 4, 5]\n"
+				+"4,[1, 3, 4, 5]\n"
+				+"5,[1, 3, 4, 5]\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationDirectionALL() throws Exception {
+
+		/*
+		 * Test that if the direction parameter OUT is given, the iteration works as expected
+		 * (i.e. it gathers information from both IN and OUT edges and neighbors
+		 * When data is gathered from the ALL edges the Gather Sum and Apply functions
+		 * set the set of vertices which are connected to a Vertex through some path as value of that vertex
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		GSAConfiguration parameters = new GSAConfiguration();
+		parameters.setDirection(EdgeDirection.ALL);
+
+		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
+
+		edges.remove(0);
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
+				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
+				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4,
+				parameters)
+				.getVertices();
+
+		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
+
+		expectedResult = "1,[1, 2, 3, 4, 5]\n"
+				+"2,[1, 2, 3, 4, 5]\n"
+				+"3,[1, 2, 3, 4, 5]\n"
+				+"4,[1, 2, 3, 4, 5]\n"
+				+"5,[1, 2, 3, 4, 5]\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class Gather extends GatherFunction<Long, Long, Long> {
+
+		@Override
+		public void preSuperstep() {
+
+			// test bcast variable
+			@SuppressWarnings("unchecked")
+			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("gatherBcastSet");
+			Assert.assertEquals(1, bcastSet.get(0));
+			Assert.assertEquals(2, bcastSet.get(1));
+			Assert.assertEquals(3, bcastSet.get(2));
+
+			// test aggregator
+			if (getSuperstepNumber() == 2) {
+				long aggrValue = ((LongValue)getPreviousIterationAggregate("superstepAggregator")).getValue();
+
+				Assert.assertEquals(7, aggrValue);
+			}
+
+			// test number of vertices
+			Assert.assertEquals(5, getNumberOfVertices());
+		}
+
+		public Long gather(Neighbor<Long, Long> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class Sum extends SumFunction<Long, Long, Long> {
+
+		LongSumAggregator aggregator = new LongSumAggregator();
+
+		@Override
+		public void preSuperstep() {
+
+			// test bcast variable
+			@SuppressWarnings("unchecked")
+			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("sumBcastSet");
+			Assert.assertEquals(4, bcastSet.get(0));
+			Assert.assertEquals(5, bcastSet.get(1));
+			Assert.assertEquals(6, bcastSet.get(2));
+
+			// test aggregator
+			aggregator = getIterationAggregator("superstepAggregator");
+
+			// test number of vertices
+			Assert.assertEquals(5, getNumberOfVertices());
+		}
+
+		public Long sum(Long newValue, Long currentValue) {
+			long superstep = getSuperstepNumber();
+			aggregator.aggregate(superstep);
+			return 0l;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class Apply extends ApplyFunction<Long, Long, Long> {
+
+		LongSumAggregator aggregator = new LongSumAggregator();
+
+		@Override
+		public void preSuperstep() {
+
+			// test bcast variable
+			@SuppressWarnings("unchecked")
+			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("applyBcastSet");
+			Assert.assertEquals(7, bcastSet.get(0));
+			Assert.assertEquals(8, bcastSet.get(1));
+			Assert.assertEquals(9, bcastSet.get(2));
+
+			// test aggregator
+			aggregator = getIterationAggregator("superstepAggregator");
+
+			// test number of vertices
+			Assert.assertEquals(5, getNumberOfVertices());
+		}
+
+		public void apply(Long summedValue, Long origValue) {
+			long superstep = getSuperstepNumber();
+			aggregator.aggregate(superstep);
+			setResult(origValue + 1);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class DummyGather extends GatherFunction<Long, Long, Long> {
+
+		@Override
+		public void preSuperstep() {
+			// test number of vertices
+			// when the numVertices option is not set, -1 is returned
+			Assert.assertEquals(-1, getNumberOfVertices());
+		}
+
+		public Long gather(Neighbor<Long, Long> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class DummySum extends SumFunction<Long, Long, Long> {
+
+		public Long sum(Long newValue, Long currentValue) {
+			return 0l;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class DummyApply extends ApplyFunction<Long, Long, Long> {
+
+		public void apply(Long summedValue, Long origValue) {
+			setResult(origValue + 1);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
+
+		public Long map(Vertex<Long, Long> value) {
+			return 1l;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class InitialiseHashSetMapper implements MapFunction<Vertex<Long, Long>, HashSet<Long>> {
+
+		@Override
+		public HashSet<Long> map(Vertex<Long, Long> value) throws Exception {
+			HashSet<Long> h = new HashSet<Long>();
+			h.add(value.getId());
+			return h;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class GetReachableVertices extends GatherFunction<HashSet<Long>, Long, HashSet<Long>> {
+
+		@Override
+		public HashSet<Long> gather(Neighbor<HashSet<Long>, Long> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class FindAllReachableVertices extends SumFunction<HashSet<Long>, Long, HashSet<Long>> {
+		@Override
+		public HashSet<Long> sum(HashSet<Long> newSet, HashSet<Long> currentSet) {
+			HashSet<Long> set = currentSet;
+			for(Long l : newSet) {
+				set.add(l);
+			}
+			return set;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class UpdateReachableVertices extends ApplyFunction<Long, HashSet<Long>, HashSet<Long>> {
+
+		@Override
+		public void apply(HashSet<Long> newValue, HashSet<Long> currentValue) {
+			newValue.addAll(currentValue);
+			if(newValue.size()>currentValue.size()) {
+				setResult(newValue);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
new file mode 100755
index 0000000..0213f02
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
@@ -0,0 +1,106 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData;
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
+import org.apache.flink.graph.library.GSAConnectedComponents;
+import org.apache.flink.graph.library.GSASingleSourceShortestPaths;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class GatherSumApplyITCase extends MultipleProgramsTestBase {
+
+	public GatherSumApplyITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	private String expectedResult;
+
+	// --------------------------------------------------------------------------------------------
+	//  Connected Components Test
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testConnectedComponents() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
+				ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env),
+				new InitMapperCC(), env);
+
+        List<Vertex<Long, Long>> result = inputGraph.run(
+        		new GSAConnectedComponents<Long, NullValue>(16)).collect();
+
+		expectedResult = "1,1\n" +
+				"2,1\n" +
+				"3,1\n" +
+				"4,1\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Single Source Shortest Path Test
+	// --------------------------------------------------------------------------------------------
+
+	@Test
+	public void testSingleSourceShortestPaths() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
+				SingleSourceShortestPathsData.getDefaultEdgeDataSet(env),
+				new InitMapperSSSP(), env);
+
+        List<Vertex<Long, Double>> result = inputGraph.run(
+        		new GSASingleSourceShortestPaths<Long>(1l, 16)).collect();
+
+		expectedResult = "1,0.0\n" +
+				"2,12.0\n" +
+				"3,13.0\n" +
+				"4,47.0\n" +
+				"5,48.0\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitMapperCC implements MapFunction<Long, Long> {
+		public Long map(Long value) {
+			return value;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitMapperSSSP implements MapFunction<Long, Double> {
+		public Double map(Long value) {
+			return 0.0;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
new file mode 100644
index 0000000..294926f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
@@ -0,0 +1,417 @@
+/*
+ * 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;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+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.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Vertex;
+
+public class TestGraphUtils {
+
+	public static final DataSet<Vertex<Long, Long>> getLongLongVertexData(
+			ExecutionEnvironment env) {
+
+		return env.fromCollection(getLongLongVertices());
+	}
+	
+	public static final DataSet<Edge<Long, Long>> getLongLongEdgeData(
+			ExecutionEnvironment env) {
+
+		return env.fromCollection(getLongLongEdges());
+	}
+
+	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidSrcData(
+			ExecutionEnvironment env) {
+		List<Edge<Long, Long>> edges = getLongLongEdges();
+
+		edges.remove(1);
+		edges.add(new Edge<Long, Long>(13L, 3L, 13L));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidTrgData(
+			ExecutionEnvironment env) {
+		List<Edge<Long, Long>> edges =  getLongLongEdges();
+
+		edges.remove(0);
+		edges.add(new Edge<Long, Long>(3L, 13L, 13L));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidSrcTrgData(
+			ExecutionEnvironment env) {
+		List<Edge<Long, Long>> edges = getLongLongEdges();
+		edges.remove(0);
+		edges.remove(1);
+		edges.remove(2);
+		edges.add(new Edge<Long, Long>(13L, 3L, 13L));
+		edges.add(new Edge<Long, Long>(1L, 12L, 12L));
+		edges.add(new Edge<Long, Long>(13L, 33L, 13L));
+		return env.fromCollection(edges);
+	}
+
+	public static final DataSet<Edge<String, Long>> getStringLongEdgeData(
+			ExecutionEnvironment env) {
+		List<Edge<String, Long>> edges = new ArrayList<Edge<String, Long>>();
+		edges.add(new Edge<String, Long>("1", "2", 12L));
+		edges.add(new Edge<String, Long>("1", "3", 13L));
+		edges.add(new Edge<String, Long>("2", "3", 23L));
+		edges.add(new Edge<String, Long>("3", "4", 34L));
+		edges.add(new Edge<String, Long>("3", "5", 35L));
+		edges.add(new Edge<String, Long>("4", "5", 45L));
+		edges.add(new Edge<String, Long>("5", "1", 51L));
+		return env.fromCollection(edges);
+	}
+
+	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2Data(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
+		tuples.add(new Tuple2<Long, Long>(1L, 10L));
+		tuples.add(new Tuple2<Long, Long>(2L, 20L));
+		tuples.add(new Tuple2<Long, Long>(3L, 30L));
+		tuples.add(new Tuple2<Long, Long>(4L, 40L));
+		tuples.add(new Tuple2<Long, Long>(6L, 60L));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2SourceData(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
+		tuples.add(new Tuple2<Long, Long>(1L, 10L));
+		tuples.add(new Tuple2<Long, Long>(1L, 20L));
+		tuples.add(new Tuple2<Long, Long>(2L, 30L));
+		tuples.add(new Tuple2<Long, Long>(3L, 40L));
+		tuples.add(new Tuple2<Long, Long>(3L, 50L));
+		tuples.add(new Tuple2<Long, Long>(4L, 60L));
+		tuples.add(new Tuple2<Long, Long>(6L, 70L));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2TargetData(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
+		tuples.add(new Tuple2<Long, Long>(2L, 10L));
+		tuples.add(new Tuple2<Long, Long>(3L, 20L));
+		tuples.add(new Tuple2<Long, Long>(3L, 30L));
+		tuples.add(new Tuple2<Long, Long>(4L, 40L));
+		tuples.add(new Tuple2<Long, Long>(6L, 50L));
+		tuples.add(new Tuple2<Long, Long>(6L, 60L));
+		tuples.add(new Tuple2<Long, Long>(1L, 70L));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple3<Long, Long, Long>> getLongLongLongTuple3Data(
+			ExecutionEnvironment env) {
+		List<Tuple3<Long, Long, Long>> tuples = new ArrayList<Tuple3<Long, Long, Long>>();
+		tuples.add(new Tuple3<Long, Long, Long>(1L, 2L, 12L));
+		tuples.add(new Tuple3<Long, Long, Long>(1L, 3L, 13L));
+		tuples.add(new Tuple3<Long, Long, Long>(2L, 3L, 23L));
+		tuples.add(new Tuple3<Long, Long, Long>(3L, 4L, 34L));
+		tuples.add(new Tuple3<Long, Long, Long>(3L, 6L, 36L));
+		tuples.add(new Tuple3<Long, Long, Long>(4L, 6L, 46L));
+		tuples.add(new Tuple3<Long, Long, Long>(6L, 1L, 61L));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2Data(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
+				DummyCustomParameterizedType<Float>>>();
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
+				new DummyCustomParameterizedType<Float>(10, 10f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
+				new DummyCustomParameterizedType<Float>(20, 20f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
+				new DummyCustomParameterizedType<Float>(30, 30f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(4L,
+				new DummyCustomParameterizedType<Float>(40, 40f)));
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2SourceData(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
+				DummyCustomParameterizedType<Float>>>();
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
+				new DummyCustomParameterizedType<Float>(10, 10f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
+				new DummyCustomParameterizedType<Float>(20, 20f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
+				new DummyCustomParameterizedType<Float>(30, 30f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
+				new DummyCustomParameterizedType<Float>(40, 40f)));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2TargetData(
+			ExecutionEnvironment env) {
+		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
+				DummyCustomParameterizedType<Float>>>();
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
+				new DummyCustomParameterizedType<Float>(10, 10f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
+				new DummyCustomParameterizedType<Float>(20, 20f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
+				new DummyCustomParameterizedType<Float>(30, 30f)));
+		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(4L,
+				new DummyCustomParameterizedType<Float>(40, 40f)));
+
+		return env.fromCollection(tuples);
+	}
+
+	public static final DataSet<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>> getLongLongCustomTuple3Data(
+			ExecutionEnvironment env) {
+		List<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>> tuples = 
+				new ArrayList<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>>();
+		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(1L, 2L,
+				new DummyCustomParameterizedType<Float>(10, 10f)));
+		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(1L, 3L,
+				new DummyCustomParameterizedType<Float>(20, 20f)));
+		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(2L, 3L,
+				new DummyCustomParameterizedType<Float>(30, 30f)));
+		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(3L, 4L,
+				new DummyCustomParameterizedType<Float>(40, 40f)));
+
+		return env.fromCollection(tuples);
+	}
+
+	/**
+	 * A graph with invalid vertex ids
+	 */
+	public static final DataSet<Vertex<Long, Long>> getLongLongInvalidVertexData(
+			ExecutionEnvironment env) {
+		List<Vertex<Long, Long>> vertices = getLongLongVertices();
+
+		vertices.remove(0);
+		vertices.add(new Vertex<Long, Long>(15L, 1L));
+
+		return env.fromCollection(vertices);
+	}
+
+	/**
+	 * A graph that has at least one vertex with no ingoing/outgoing edges
+	 */
+	public static final DataSet<Edge<Long, Long>> getLongLongEdgeDataWithZeroDegree(
+			ExecutionEnvironment env) {
+		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+		edges.add(new Edge<Long, Long>(1L, 2L, 12L));
+		edges.add(new Edge<Long, Long>(1L, 4L, 14L));
+		edges.add(new Edge<Long, Long>(1L, 5L, 15L));
+		edges.add(new Edge<Long, Long>(2L, 3L, 23L));
+		edges.add(new Edge<Long, Long>(3L, 5L, 35L));
+		edges.add(new Edge<Long, Long>(4L, 5L, 45L));
+
+		return env.fromCollection(edges);
+	}
+
+	/**
+	 * Function that produces an ArrayList of vertices
+	 */
+	public static final List<Vertex<Long, Long>> getLongLongVertices() {
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(1L, 1L));
+		vertices.add(new Vertex<Long, Long>(2L, 2L));
+		vertices.add(new Vertex<Long, Long>(3L, 3L));
+		vertices.add(new Vertex<Long, Long>(4L, 4L));
+		vertices.add(new Vertex<Long, Long>(5L, 5L));
+
+		return vertices;
+	}
+
+	public static final List<Vertex<Long, Boolean>> getLongBooleanVertices() {
+		List<Vertex<Long, Boolean>> vertices = new ArrayList<Vertex<Long, Boolean>>();
+		vertices.add(new Vertex<Long, Boolean>(1L, true));
+		vertices.add(new Vertex<Long, Boolean>(2L, true));
+		vertices.add(new Vertex<Long, Boolean>(3L, true));
+		vertices.add(new Vertex<Long, Boolean>(4L, true));
+		vertices.add(new Vertex<Long, Boolean>(5L, true));
+
+		return vertices;
+	}
+
+	public static final DataSet<Edge<Long, Long>> getDisconnectedLongLongEdgeData(
+				ExecutionEnvironment env) {
+			List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+			edges.add(new Edge<Long, Long>(1L, 2L, 12L));
+			edges.add(new Edge<Long, Long>(1L, 3L, 13L));
+			edges.add(new Edge<Long, Long>(2L, 3L, 23L));
+			edges.add(new Edge<Long, Long>(4L, 5L, 45L));
+			
+			return env.fromCollection(edges);
+		}
+	
+	/**
+	 * Function that produces an ArrayList of edges
+	 */
+	public static final List<Edge<Long, Long>> getLongLongEdges() {
+		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+		edges.add(new Edge<Long, Long>(1L, 2L, 12L));
+		edges.add(new Edge<Long, Long>(1L, 3L, 13L));
+		edges.add(new Edge<Long, Long>(2L, 3L, 23L));
+		edges.add(new Edge<Long, Long>(3L, 4L, 34L));
+		edges.add(new Edge<Long, Long>(3L, 5L, 35L));
+		edges.add(new Edge<Long, Long>(4L, 5L, 45L));
+		edges.add(new Edge<Long, Long>(5L, 1L, 51L));
+	
+		return edges;
+	}
+
+	public static class DummyCustomType implements Serializable {
+		private static final long serialVersionUID = 1L;
+		
+		private int intField;
+		private boolean booleanField;
+		
+		public DummyCustomType(int intF, boolean boolF) {
+			this.intField = intF;
+			this.booleanField = boolF;
+		}
+		
+		public DummyCustomType() {
+			this.intField = 0;
+			this.booleanField = true;
+		}
+
+		public int getIntField() {
+			return intField;
+		}
+		
+		public void setIntField(int intF) {
+			this.intField = intF;
+		}
+		
+		public boolean getBooleanField() {
+			return booleanField;
+		}
+		
+		@Override
+		public String toString() {
+			return booleanField ? "(T," + intField + ")" : "(F," + intField + ")";
+		}
+	}
+	
+	public static class DummyCustomParameterizedType<T> implements Serializable {
+		private static final long serialVersionUID = 1L;
+		
+		private int intField;
+		private T tField;
+		
+		public DummyCustomParameterizedType(int intF, T tF) {
+			this.intField = intF;
+			this.tField = tF;
+		}
+		
+		public DummyCustomParameterizedType() {
+			this.intField = 0;
+			this.tField = null;
+		}
+
+		public int getIntField() {
+			return intField;
+		}
+		
+		public void setIntField(int intF) {
+			this.intField = intF;
+		}
+		
+		public void setTField(T tF) {
+			this.tField = tF;
+		}
+		
+		public T getTField() {
+			return tField;
+		}
+		
+		@Override
+		public String toString() {
+			return "(" + tField.toString() + "," + intField + ")";
+		}
+	}
+
+	/**
+	 * Method useful for suppressing sysout printing
+	 */
+	public static void pipeSystemOutToNull() {
+		System.setOut(new PrintStream(new BlackholeOutputSteam()));
+	}
+
+	private static final class BlackholeOutputSteam extends java.io.OutputStream {
+		@Override
+		public void write(int b){}
+	}
+
+	/**
+	 * utils for getting the second graph for the test of method difference();
+	 * @param env
+	 */
+	public static final DataSet<Edge<Long,Long>> getLongLongEdgeDataDifference(
+			ExecutionEnvironment env){
+		return env.fromCollection(getLongLongEdgesForDifference());
+	}
+
+	public static final DataSet<Edge<Long,Long>> getLongLongEdgeDataDifference2(
+			ExecutionEnvironment env){
+		return env.fromCollection(getLongLongEdgesForDifference2());
+	}
+
+	public static final DataSet<Vertex<Long,Long>> getLongLongVertexDataDifference(
+			ExecutionEnvironment env)
+	{
+		return env.fromCollection(getVerticesForDifference());
+	}
+
+	public static final List<Vertex<Long,Long>> getVerticesForDifference(){
+		List<Vertex<Long,Long>> vertices = new ArrayList<Vertex<Long,Long>>();
+		vertices.add(new Vertex<Long, Long>(1L, 1L));
+		vertices.add(new Vertex<Long, Long>(3L, 3L));
+		vertices.add(new Vertex<Long, Long>(6L, 6L));
+
+		return vertices;
+
+	}
+
+	public static final List<Edge<Long, Long>> getLongLongEdgesForDifference() {
+		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+		edges.add(new Edge<Long, Long>(1L, 3L, 13L));
+		edges.add(new Edge<Long, Long>(1L, 6L, 26L));
+		edges.add(new Edge<Long, Long>(6L, 3L, 63L));
+		return edges;
+	}
+
+	public static final List<Edge<Long, Long>> getLongLongEdgesForDifference2() {
+		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+		edges.add(new Edge<Long, Long>(6L, 6L, 66L));
+		return edges;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
new file mode 100644
index 0000000..0feb3fb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
@@ -0,0 +1,689 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+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.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexCentricConfiguration;
+import org.apache.flink.graph.spargel.VertexCentricIteration;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.LongValue;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.apache.flink.graph.utils.VertexToTuple2Map;
+
+@RunWith(Parameterized.class)
+public class VertexCentricConfigurationITCase extends MultipleProgramsTestBase {
+
+	public VertexCentricConfigurationITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testRunWithConfiguration() throws Exception {
+		/*
+		 * Test Graph's runVertexCentricIteration when configuration parameters are provided
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
+				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
+
+		// create the configuration object
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.addBroadcastSetForUpdateFunction("updateBcastSet", env.fromElements(1, 2, 3));
+		parameters.addBroadcastSetForMessagingFunction("messagingBcastSet", env.fromElements(4, 5, 6));
+		parameters.registerAggregator("superstepAggregator", new LongSumAggregator());
+		parameters.setOptNumVertices(true);
+
+		Graph<Long, Long, Long> res = graph.runVertexCentricIteration(
+				new UpdateFunction(), new MessageFunction(), 10, parameters);
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+        
+		expectedResult = "1,11\n" +
+						"2,11\n" +
+						"3,11\n" +
+						"4,11\n" +
+						"5,11";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationConfiguration() throws Exception {
+
+		/*
+		 * Test name, parallelism and solutionSetUnmanaged parameters
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		VertexCentricIteration<Long, Long, Long, Long> iteration = VertexCentricIteration
+				.withEdges(TestGraphUtils.getLongLongEdgeData(env), new DummyUpdateFunction(), 
+						new DummyMessageFunction(), 10);
+		
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+		parameters.setName("gelly iteration");
+		parameters.setParallelism(2);
+		parameters.setSolutionSetUnmanagedMemory(true);
+		
+		iteration.configure(parameters);
+		
+		Assert.assertEquals("gelly iteration", iteration.getIterationConfiguration().getName(""));
+		Assert.assertEquals(2, iteration.getIterationConfiguration().getParallelism());
+		Assert.assertEquals(true, iteration.getIterationConfiguration().isSolutionSetUnmanagedMemory());
+
+		DataSet<Vertex<Long, Long>> data = TestGraphUtils.getLongLongVertexData(env).runOperation(iteration);
+        List<Vertex<Long,Long>> result= data.collect();
+        
+		expectedResult = "1,11\n" +
+						"2,12\n" +
+						"3,13\n" +
+						"4,14\n" +
+						"5,15";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testDefaultConfiguration() throws Exception {
+		/*
+		 * Test Graph's runVertexCentricIteration when configuration parameters are not provided
+		 * i.e. degrees and numVertices will be -1, EdgeDirection will be OUT.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
+				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
+
+		Graph<Long, Long, Long> res = graph.runVertexCentricIteration(
+				new UpdateFunctionDefault(), new MessageFunctionDefault(), 5);
+
+		
+		DataSet<Tuple2<Long, Long>> data = res.getVertices().map(new VertexToTuple2Map<Long, Long>());
+        List<Tuple2<Long, Long>> result= data.collect();
+        
+		expectedResult = "1,6\n" +
+						"2,6\n" +
+						"3,6\n" +
+						"4,6\n" +
+						"5,6";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationDefaultDirection() throws Exception {
+
+		/*
+		 * Test that if no direction parameter is given, the iteration works as before
+		 * (i.e. it collects messages from the in-neighbors and sends them to the out-neighbors)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
+				.mapVertices(new InitialiseHashSetMapper());
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
+				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerTrg(), 5)
+				.getVertices();
+
+        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
+
+		expectedResult = "1,[5]\n" +
+				"2,[1]\n" +
+				"3,[1, 2]\n" +
+				"4,[3]\n" +
+				"5,[3, 4]";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationINDirection() throws Exception {
+
+		/*
+		 * Test that if the direction parameter is set to IN,
+		 * messages are collected from the out-neighbors and sent to the in-neighbors.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
+				.mapVertices(new InitialiseHashSetMapper());
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.setDirection(EdgeDirection.IN);
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
+				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerSrc(), 5, parameters)
+				.getVertices();
+
+        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
+
+		expectedResult = "1,[2, 3]\n" +
+				"2,[3]\n" +
+				"3,[4, 5]\n" +
+				"4,[5]\n" +
+				"5,[1]";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testIterationALLDirection() throws Exception {
+
+		/*
+		 * Test that if the direction parameter is set to ALL,
+		 * messages are collected from all the neighbors and sent to all the neighbors.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, HashSet<Long>, Long> graph = Graph
+				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
+				.mapVertices(new InitialiseHashSetMapper());
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.setDirection(EdgeDirection.ALL);
+
+		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
+				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerAll(), 5, parameters)
+				.getVertices();
+
+        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
+
+		expectedResult = "1,[2, 3, 5]\n" +
+				"2,[1, 3]\n" +
+				"3,[1, 2, 4, 5]\n" +
+				"4,[3, 5]\n" +
+				"5,[1, 3, 4]";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testNumVerticesNotSet() throws Exception {
+
+		/*
+		 * Test that if the number of vertices option is not set, -1 is returned as value.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		DataSet<Vertex<Long, Long>> verticesWithNumVertices = graph.runVertexCentricIteration(new UpdateFunctionNumVertices(),
+				new DummyMessageFunction(), 2).getVertices();
+
+        List<Vertex<Long, Long>> result= verticesWithNumVertices.collect();
+
+		expectedResult = "1,-1\n" +
+				"2,-1\n" +
+				"3,-1\n" +
+				"4,-1\n" +
+				"5,-1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testInDegreesSet() throws Exception {
+
+		/*
+		 * Test that if the degrees are set, they can be accessed in every superstep 
+		 * inside the update function and the value
+		 * is correctly computed for degrees in the messaging function.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.setOptDegrees(true);
+
+		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
+				new UpdateFunctionInDegrees(), new DegreesMessageFunction(), 5, parameters).getVertices();
+
+        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
+
+		expectedResult = "1,1\n" +
+				"2,1\n" +
+				"3,2\n" +
+				"4,1\n" +
+				"5,2";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testInDegreesNotSet() throws Exception {
+
+		/*
+		 * Test that if the degrees option is not set, then -1 is returned as a value for in-degree.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
+				new UpdateFunctionInDegrees(), new DummyMessageFunction(), 2).getVertices();
+
+        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
+
+		expectedResult = "1,-1\n" +
+				"2,-1\n" +
+				"3,-1\n" +
+				"4,-1\n" +
+				"5,-1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testOutDegreesSet() throws Exception {
+
+		/*
+		 * Test that if the degrees are set, they can be accessed in every superstep
+		 * inside the update function and the value
+		 * is correctly computed for degrees in the messaging function.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.setOptDegrees(true);
+
+		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
+				new UpdateFunctionOutDegrees(), new DegreesMessageFunction(), 5, parameters).getVertices();
+
+        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
+
+		expectedResult = "1,2\n" +
+				"2,1\n" +
+				"3,2\n" +
+				"4,1\n" +
+				"5,1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testOutDegreesNotSet() throws Exception {
+
+		/*
+		 * Test that if the degrees option is not set, then -1 is returned as a value for out-degree.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
+				new UpdateFunctionInDegrees(), new DummyMessageFunction(), 2).getVertices();
+
+        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
+
+		expectedResult = "1,-1\n" +
+				"2,-1\n" +
+				"3,-1\n" +
+				"4,-1\n" +
+				"5,-1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testDirectionALLAndDegrees() throws Exception {
+
+		/*
+		 * Compute the number of neighbors in a vertex - centric manner, and verify that it is equal to
+		 * the sum: inDegree + outDegree.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Boolean, Long> graph = Graph.fromCollection(TestGraphUtils.getLongBooleanVertices(),
+				TestGraphUtils.getLongLongEdges(), env);
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		parameters.setOptDegrees(true);
+		parameters.setDirection(EdgeDirection.ALL);
+
+		DataSet<Vertex<Long, Boolean>> verticesWithNumNeighbors = graph.runVertexCentricIteration(
+				new VertexUpdateNumNeighbors(), new IdMessenger(), 1, parameters).getVertices();
+
+        List<Vertex<Long, Boolean>> result= verticesWithNumNeighbors.collect();
+
+		expectedResult = "1,true\n" +
+				"2,true\n" +
+				"3,true\n" +
+				"4,true\n" +
+				"5,true";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
+
+		LongSumAggregator aggregator = new LongSumAggregator();
+
+		@Override
+		public void preSuperstep() {
+			
+			// test bcast variable
+			@SuppressWarnings("unchecked")
+			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("updateBcastSet");
+			Assert.assertEquals(1, bcastSet.get(0));
+			Assert.assertEquals(2, bcastSet.get(1));
+			Assert.assertEquals(3, bcastSet.get(2));
+			
+			// test aggregator
+			aggregator = getIterationAggregator("superstepAggregator");
+
+			// test number of vertices
+			Assert.assertEquals(5, getNumberOfVertices());
+			
+		}
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			long superstep = getSuperstepNumber();
+			aggregator.aggregate(superstep);
+
+			setNewVertexValue(vertex.getValue() + 1);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunctionDefault extends VertexUpdateFunction<Long, Long, Long> {
+
+		LongSumAggregator aggregator = new LongSumAggregator();
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+
+			// test number of vertices
+			Assert.assertEquals(-1, getNumberOfVertices());
+
+			// test degrees
+			Assert.assertEquals(-1, getInDegree());
+			Assert.assertEquals(-1, getOutDegree());
+
+			setNewVertexValue(vertex.getValue() + 1);
+		}
+	}
+	
+	@SuppressWarnings("serial")
+	public static final class MessageFunction extends MessagingFunction<Long, Long, Long, Long> {
+
+		@Override
+		public void preSuperstep() {
+			
+			// test bcast variable
+			@SuppressWarnings("unchecked")
+			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("messagingBcastSet");
+			Assert.assertEquals(4, bcastSet.get(0));
+			Assert.assertEquals(5, bcastSet.get(1));
+			Assert.assertEquals(6, bcastSet.get(2));
+
+			// test number of vertices
+			Assert.assertEquals(5, getNumberOfVertices());
+			
+			// test aggregator
+			if (getSuperstepNumber() == 2) {
+				long aggrValue = ((LongValue)getPreviousIterationAggregate("superstepAggregator")).getValue();
+				Assert.assertEquals(5, aggrValue);
+			}
+		}
+
+		@Override
+		public void sendMessages(Vertex<Long, Long> vertex) {
+			//send message to keep vertices active
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class MessageFunctionDefault extends MessagingFunction<Long, Long, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, Long> vertex) {
+			// test number of vertices
+			Assert.assertEquals(-1, getNumberOfVertices());
+
+			// test degrees
+			Assert.assertEquals(-1, getInDegree());
+			Assert.assertEquals(-1, getOutDegree());
+			//send message to keep vertices active
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunctionNumVertices extends VertexUpdateFunction<Long, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+				setNewVertexValue(getNumberOfVertices());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class DummyUpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			setNewVertexValue(vertex.getValue() + 1);
+		}
+	}
+	
+	@SuppressWarnings("serial")
+	public static final class DummyMessageFunction extends MessagingFunction<Long, Long, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, Long> vertex) {
+			//send message to keep vertices active
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class DegreesMessageFunction extends MessagingFunction<Long, Long, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, Long> vertex) {
+			if (vertex.getId().equals(1)) {
+				Assert.assertEquals(2, getOutDegree());
+				Assert.assertEquals(1, getInDegree());
+			}
+			else if(vertex.getId().equals(3)) {
+				Assert.assertEquals(2, getOutDegree());
+				Assert.assertEquals(2, getInDegree());
+			}
+			//send message to keep vertices active
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class VertexUpdateDirection extends VertexUpdateFunction<Long, HashSet<Long>, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, HashSet<Long>> vertex, MessageIterator<Long> messages) throws Exception {
+			vertex.getValue().clear();
+
+			for(long msg : messages) {
+				vertex.getValue().add(msg);
+			}
+
+			setNewVertexValue(vertex.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunctionInDegrees extends VertexUpdateFunction<Long, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			long inDegree = getInDegree();
+			setNewVertexValue(inDegree);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunctionOutDegrees extends VertexUpdateFunction<Long, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			long outDegree = getOutDegree();
+			setNewVertexValue(outDegree);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class VertexUpdateNumNeighbors extends VertexUpdateFunction<Long, Boolean,
+			Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Boolean> vertex, MessageIterator<Long> messages) throws Exception {
+
+			long count = 0;
+
+			for(@SuppressWarnings("unused") long msg : messages) {
+				count++;
+			}
+			setNewVertexValue(count == (getInDegree() + getOutDegree()));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class UpdateFunctionDegrees extends VertexUpdateFunction<Long, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
+			long inDegree = getInDegree();
+			long outDegree = getOutDegree();
+			setNewVertexValue(inDegree + outDegree);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class IdMessengerSrc extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
+			for (Edge<Long, Long> edge : getEdges()) {
+				sendMessageTo(edge.getSource(), vertex.getId());
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class IdMessengerAll extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
+			for (Edge<Long, Long> edge : getEdges()) {
+				if(edge.getSource() != vertex.getId()) {
+					sendMessageTo(edge.getSource(), vertex.getId());
+				} else {
+					sendMessageTo(edge.getTarget(), vertex.getId());
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class IdMessenger extends MessagingFunction<Long, Boolean, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, Boolean> vertex) throws Exception {
+			for (Edge<Long, Long> edge : getEdges()) {
+				if(edge.getSource() != vertex.getId()) {
+					sendMessageTo(edge.getSource(), vertex.getId());
+				} else {
+					sendMessageTo(edge.getTarget(), vertex.getId());
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class IdMessengerTrg extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
+
+		@Override
+		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
+			for (Edge<Long, Long> edge : getEdges()) {
+				sendMessageTo(edge.getTarget(), vertex.getId());
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
+
+		public Long map(Vertex<Long, Long> value) {
+			return 1l;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class InitialiseHashSetMapper implements MapFunction<Vertex<Long, Long>, HashSet<Long>> {
+
+		@Override
+		public HashSet<Long> map(Vertex<Long, Long> value) throws Exception {
+			return new HashSet<Long>();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java
new file mode 100644
index 0000000..b0bacc4
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.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.test.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.flink.graph.example.ConnectedComponents;
+import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+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 {
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
new file mode 100644
index 0000000..183c429
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
@@ -0,0 +1,77 @@
+/*
+ * 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.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.flink.graph.example.EuclideanGraphWeighing;
+import org.apache.flink.graph.example.utils.EuclideanGraphData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+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 EuclideanGraphWeighingITCase extends MultipleProgramsTestBase {
+
+	private String verticesPath;
+
+	private String edgesPath;
+
+	private String resultPath;
+
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public EuclideanGraphWeighingITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Before
+	public void before() throws Exception {
+		resultPath = tempFolder.newFile().toURI().toString();
+		File verticesFile = tempFolder.newFile();
+		Files.write(EuclideanGraphData.VERTICES, verticesFile, Charsets.UTF_8);
+
+		File edgesFile = tempFolder.newFile();
+		Files.write(EuclideanGraphData.EDGES, edgesFile, Charsets.UTF_8);
+
+		verticesPath = verticesFile.toURI().toString();
+		edgesPath = edgesFile.toURI().toString();
+	}
+
+	@Test
+	public void testGraphWeightingWeighing() throws Exception {
+		EuclideanGraphWeighing.main(new String[]{verticesPath, edgesPath, resultPath});
+		expected = EuclideanGraphData.RESULTED_WEIGHTED_EDGES;
+	}
+
+	@After
+	public void after() throws Exception {
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
new file mode 100644
index 0000000..c19411b
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
@@ -0,0 +1,133 @@
+/*
+ * 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.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.IncrementalSSSP;
+import org.apache.flink.graph.example.utils.IncrementalSSSPData;
+import org.apache.flink.graph.spargel.VertexCentricConfiguration;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+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 IncrementalSSSPITCase extends MultipleProgramsTestBase {
+
+	private String verticesPath;
+
+	private String edgesPath;
+
+	private String edgesInSSSPPath;
+
+	private String resultPath;
+
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public IncrementalSSSPITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Before
+	public void before() throws Exception {
+		resultPath = tempFolder.newFile().toURI().toString();
+		File verticesFile = tempFolder.newFile();
+		Files.write(IncrementalSSSPData.VERTICES, verticesFile, Charsets.UTF_8);
+
+		File edgesFile = tempFolder.newFile();
+		Files.write(IncrementalSSSPData.EDGES, edgesFile, Charsets.UTF_8);
+
+		File edgesInSSSPFile = tempFolder.newFile();
+		Files.write(IncrementalSSSPData.EDGES_IN_SSSP, edgesInSSSPFile, Charsets.UTF_8);
+
+		verticesPath = verticesFile.toURI().toString();
+		edgesPath = edgesFile.toURI().toString();
+		edgesInSSSPPath = edgesInSSSPFile.toURI().toString();
+	}
+
+	@Test
+	public void testIncrementalSSSP() throws Exception {
+		IncrementalSSSP.main(new String[]{verticesPath, edgesPath, edgesInSSSPPath,
+				IncrementalSSSPData.SRC_EDGE_TO_BE_REMOVED, IncrementalSSSPData.TRG_EDGE_TO_BE_REMOVED,
+				IncrementalSSSPData.VAL_EDGE_TO_BE_REMOVED,resultPath, IncrementalSSSPData.NUM_VERTICES + ""});
+		expected = IncrementalSSSPData.RESULTED_VERTICES;
+	}
+
+	@Test
+	public void testIncrementalSSSPNonSPEdge() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Vertex<Long, Double>> vertices = IncrementalSSSPData.getDefaultVertexDataSet(env);
+		DataSet<Edge<Long, Double>> edges = IncrementalSSSPData.getDefaultEdgeDataSet(env);
+		DataSet<Edge<Long, Double>> edgesInSSSP = IncrementalSSSPData.getDefaultEdgesInSSSP(env);
+		// the edge to be removed is a non-SP edge
+		Edge<Long, Double> edgeToBeRemoved = new Edge<Long, Double>(3L, 5L, 5.0);
+
+		Graph<Long, Double, Double> graph = Graph.fromDataSet(vertices, edges, env);
+		// Assumption: all minimum weight paths are kept
+		Graph<Long, Double, Double> ssspGraph = Graph.fromDataSet(vertices, edgesInSSSP, env);
+		// remove the edge
+		graph.removeEdge(edgeToBeRemoved);
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		if(IncrementalSSSP.isInSSSP(edgeToBeRemoved, edgesInSSSP)) {
+
+			parameters.setDirection(EdgeDirection.IN);
+			parameters.setOptDegrees(true);
+
+			// run the vertex centric iteration to propagate info
+			Graph<Long, Double, Double> result = ssspGraph.runVertexCentricIteration(
+					new IncrementalSSSP.VertexDistanceUpdater(),
+					new IncrementalSSSP.InvalidateMessenger(edgeToBeRemoved),
+					IncrementalSSSPData.NUM_VERTICES, parameters);
+
+			DataSet<Vertex<Long, Double>> resultedVertices = result.getVertices();
+
+			resultedVertices.writeAsCsv(resultPath, "\n", ",");
+			env.execute();
+		} else {
+			vertices.writeAsCsv(resultPath, "\n", ",");
+			env.execute();
+		}
+
+		expected = IncrementalSSSPData.VERTICES;
+	}
+
+	@After
+	public void after() throws Exception {
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java
new file mode 100644
index 0000000..294a756
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java
@@ -0,0 +1,72 @@
+/*
+ * 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.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.flink.graph.example.JaccardSimilarityMeasure;
+import org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+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 JaccardSimilarityMeasureITCase extends MultipleProgramsTestBase {
+
+	private String edgesPath;
+
+	private String resultPath;
+
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public JaccardSimilarityMeasureITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Before
+	public void before() throws Exception {
+		resultPath = tempFolder.newFile().toURI().toString();
+
+		File edgesFile = tempFolder.newFile();
+		Files.write(JaccardSimilarityMeasureData.EDGES, edgesFile, Charsets.UTF_8);
+
+		edgesPath = edgesFile.toURI().toString();
+	}
+
+	@Test
+	public void testJaccardSimilarityMeasureExample() throws Exception {
+		JaccardSimilarityMeasure.main(new String[]{edgesPath, resultPath});
+		expected = JaccardSimilarityMeasureData.JACCARD_EDGES;
+	}
+
+	@After
+	public void after() throws Exception {
+		compareResultsByLinesInMemory(expected, resultPath);
+	}
+}


[20/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
new file mode 100644
index 0000000..f64c701
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
@@ -0,0 +1,486 @@
+/*
+ * 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;
+import com.google.common.base.Preconditions;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.io.CsvReader;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * A class to build a Graph using path(s) provided to CSV file(s) with optional vertex and edge data.
+ * The class also configures the CSV readers used to read edge and vertex data such as the field types,
+ * the delimiters (row and field), the fields that should be included or skipped, and other flags,
+ * such as whether to skip the initial line as the header.
+ * The configuration is done using the functions provided in the {@link org.apache.flink.api.java.io.CsvReader} class.
+ */
+
+public class GraphCsvReader {
+
+	@SuppressWarnings("unused")
+	private final Path vertexPath, edgePath;
+	private final ExecutionEnvironment executionContext;
+	protected CsvReader edgeReader;
+	protected CsvReader vertexReader;
+	protected MapFunction<?, ?> mapper;
+	protected Class<?> vertexKey;
+	protected Class<?> vertexValue;
+	protected Class<?> edgeValue;
+
+//--------------------------------------------------------------------------------------------------------------------
+	public GraphCsvReader(Path vertexPath, Path edgePath, ExecutionEnvironment context) {
+		this.vertexPath = vertexPath;
+		this.edgePath = edgePath;
+		this.vertexReader = new CsvReader(vertexPath, context);
+		this.edgeReader = new CsvReader(edgePath, context);
+		this.mapper = null;
+		this.executionContext = context;
+	}
+
+	public GraphCsvReader(Path edgePath, ExecutionEnvironment context) {
+		this.vertexPath = null;
+		this.edgePath = edgePath;
+		this.edgeReader = new CsvReader(edgePath, context);
+		this.vertexReader = null;
+		this.mapper = null;
+		this.executionContext = context;
+	}
+
+	public <K, VV> GraphCsvReader(Path edgePath, final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+		this.vertexPath = null;
+		this.edgePath = edgePath;
+		this.edgeReader = new CsvReader(edgePath, context);
+		this.vertexReader = null;
+		this.mapper = mapper;
+		this.executionContext = context;
+	}
+
+	public GraphCsvReader (String edgePath, ExecutionEnvironment context) {
+		this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context);
+
+	}
+
+	public GraphCsvReader(String vertexPath, String edgePath, ExecutionEnvironment context) {
+		this(new Path(Preconditions.checkNotNull(vertexPath, "The file path may not be null.")),
+				new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context);
+	}
+
+
+	public <K, VV> GraphCsvReader(String edgePath, final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+			this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), mapper, context);
+	}
+
+	/**
+	 * Creates a Graph from CSV input with vertex values and edge values.
+	 * The vertex values are specified through a vertices input file or a user-defined map function.
+	 * 
+	 * @param vertexKey the type of the vertex IDs
+	 * @param vertexValue the type of the vertex values
+	 * @param edgeValue the type of the edge values
+	 * @return a Graph with vertex and edge values.
+	 */
+	@SuppressWarnings("unchecked")
+	public <K, VV, EV> Graph<K, VV, EV> types(Class<K> vertexKey, Class<VV> vertexValue,
+			Class<EV> edgeValue) {
+
+		DataSet<Tuple2<K, VV>> vertices = null;
+
+		if (edgeReader == null) {
+			throw new RuntimeException("The edges input file cannot be null!");
+		}
+
+		DataSet<Tuple3<K, K, EV>> edges = edgeReader.types(vertexKey, vertexKey, edgeValue);
+
+		// the vertex value can be provided by an input file or a user-defined mapper
+		if (vertexReader != null) {
+			vertices = vertexReader.types(vertexKey, vertexValue);
+			return Graph.fromTupleDataSet(vertices, edges, executionContext);
+		}
+		else if (mapper != null) {
+			return Graph.fromTupleDataSet(edges, (MapFunction<K, VV>) mapper, executionContext);
+		}
+		else {
+			throw new RuntimeException("Vertex values have to be specified through a vertices input file"
+					+ "or a user-defined map function.");
+		}
+	}
+
+	/**
+	 * Creates a Graph from CSV input with edge values, but without vertex values.
+	 * @param vertexKey the type of the vertex IDs
+	 * @param edgeValue the type of the edge values
+	 * @return a Graph where the edges are read from an edges CSV file (with values).
+	 */
+	public <K, EV> Graph<K, NullValue, EV> edgeTypes(Class<K> vertexKey, Class<EV> edgeValue) {
+
+		if (edgeReader == null) {
+			throw new RuntimeException("The edges input file cannot be null!");
+		}
+
+		DataSet<Tuple3<K, K, EV>> edges = edgeReader.types(vertexKey, vertexKey, edgeValue);
+
+		return Graph.fromTupleDataSet(edges, executionContext);
+	}
+
+	/**
+	 * Creates a Graph from CSV input without vertex values or edge values.
+	 * @param vertexKey the type of the vertex IDs
+	 * @return a Graph where the vertex IDs are read from the edges input file.
+	 */
+	public <K> Graph<K, NullValue, NullValue> keyType(Class<K> vertexKey) {
+
+		if (edgeReader == null) {
+			throw new RuntimeException("The edges input file cannot be null!");
+		}
+
+		@SuppressWarnings("serial")
+		DataSet<Tuple3<K, K, NullValue>> edges = edgeReader.types(vertexKey, vertexKey)
+				.map(new MapFunction<Tuple2<K, K>, Tuple3<K, K, NullValue>>() {
+
+					public Tuple3<K, K, NullValue> map(Tuple2<K, K> edge) {
+						return new Tuple3<K, K, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
+					}
+				}).withForwardedFields("f0;f1");;
+
+		return Graph.fromTupleDataSet(edges, executionContext);
+	}
+
+	/**
+	 * Creates a Graph from CSV input without edge values.
+	 * The vertex values are specified through a vertices input file or a user-defined map function.
+	 * If no vertices input file is provided, the vertex IDs are automatically created from the edges
+	 * input file.
+	 * @param vertexKey the type of the vertex IDs
+	 * @param vertexValue the type of the vertex values
+	 * @return a Graph where the vertex IDs and vertex values.
+	 */
+	@SuppressWarnings({ "serial", "unchecked" })
+	public <K, VV> Graph<K, VV, NullValue> vertexTypes(Class<K> vertexKey, Class<VV> vertexValue) {
+		
+		DataSet<Tuple2<K, VV>> vertices = null;
+
+		if (edgeReader == null) {
+			throw new RuntimeException("The edges input file cannot be null!");
+		}
+
+		DataSet<Tuple3<K, K, NullValue>> edges = edgeReader.types(vertexKey, vertexKey)
+				.map(new MapFunction<Tuple2<K,K>, Tuple3<K, K, NullValue>>() {
+
+					public Tuple3<K, K, NullValue> map(Tuple2<K, K> input) {
+						return new Tuple3<K, K, NullValue>(input.f0, input.f1, NullValue.getInstance());
+					}
+				}).withForwardedFields("f0;f1");
+
+		// the vertex value can be provided by an input file or a user-defined mapper
+		if (vertexReader != null) {
+			vertices = vertexReader.types(vertexKey, vertexValue);
+			return Graph.fromTupleDataSet(vertices, edges, executionContext);
+		}
+		else if (mapper != null) {
+			return Graph.fromTupleDataSet(edges, (MapFunction<K, VV>) mapper, executionContext);
+		}
+		else {
+			throw new RuntimeException("Vertex values have to be specified through a vertices input file"
+					+ "or a user-defined map function.");
+		}
+	}
+
+	/**
+	 *Configures the Delimiter that separates rows for the CSV reader used to read the edges
+	 *	({@code '\n'}) is used by default.
+	 *
+	 *@param delimiter The delimiter that separates the rows.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader lineDelimiterEdges(String delimiter) {
+		edgeReader.lineDelimiter(delimiter);
+		return this;
+	}
+
+	/**
+	 *Configures the Delimiter that separates rows for the CSV reader used to read the vertices
+	 *	({@code '\n'}) is used by default.
+	 *
+	 *@param delimiter The delimiter that separates the rows.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader lineDelimiterVertices(String delimiter) {
+		if(this.vertexReader != null) {
+			this.vertexReader.lineDelimiter(delimiter);
+		}
+		return this;
+	}
+
+	/**
+	 *Configures the Delimiter that separates fields in a row for the CSV reader used to read the vertices
+	 * ({@code ','}) is used by default.
+	 *
+	 * @param delimiter The delimiter that separates the fields in a row.
+	 * @return The GraphCsv reader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader fieldDelimiterVertices(String delimiter) {
+		if(this.vertexReader != null) {
+			this.vertexReader.fieldDelimiter(delimiter);
+		}
+		return this;
+	}
+
+	/**
+	 *Configures the Delimiter that separates fields in a row for the CSV reader used to read the edges
+	 * ({@code ','}) is used by default.
+	 *
+	 * @param delimiter The delimiter that separates the fields in a row.
+	 * @return The GraphCsv reader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader fieldDelimiterEdges(String delimiter) {
+		this.edgeReader.fieldDelimiter(delimiter);
+		return this;
+	}
+
+	/**
+	 * Enables quoted String parsing for Edge Csv Reader. Field delimiters in quoted Strings are ignored.
+	 * A String is parsed as quoted if it starts and ends with a quoting character and as unquoted otherwise.
+	 * Leading or tailing whitespaces are not allowed.
+	 *
+	 * @param quoteCharacter The character which is used as quoting character.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader parseQuotedStringsEdges(char quoteCharacter) {
+		this.edgeReader.parseQuotedStrings(quoteCharacter);
+		return this;
+	}
+
+	/**
+	 * Enables quoted String parsing for Vertex Csv Reader. Field delimiters in quoted Strings are ignored.
+	 * A String is parsed as quoted if it starts and ends with a quoting character and as unquoted otherwise.
+	 * Leading or tailing whitespaces are not allowed.
+	 *
+	 * @param quoteCharacter The character which is used as quoting character.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader parseQuotedStringsVertices(char quoteCharacter) {
+		if(this.vertexReader != null) {
+			this.vertexReader.parseQuotedStrings(quoteCharacter);
+		}
+		return this;
+	}
+
+	/**
+	 * Configures the string that starts comments for the Vertex Csv Reader.
+	 * By default comments will be treated as invalid lines.
+	 * This function only recognizes comments which start at the beginning of the line!
+	 *
+	 * @param commentPrefix The string that starts the comments.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreCommentsVertices(String commentPrefix) {
+		if(this.vertexReader != null) {
+			this.vertexReader.ignoreComments(commentPrefix);
+		}
+		return this;
+	}
+
+	/**
+	 * Configures the string that starts comments for the Edge Csv Reader.
+	 * By default comments will be treated as invalid lines.
+	 * This function only recognizes comments which start at the beginning of the line!
+	 *
+	 * @param commentPrefix The string that starts the comments.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreCommentsEdges(String commentPrefix) {
+		this.edgeReader.ignoreComments(commentPrefix);
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
+	 * parser will look at the first {@code n} fields, where {@code n} is the length of the boolean
+	 * array. The parser will skip over all fields where the boolean value at the corresponding position
+	 * in the array is {@code false}. The result contains the fields where the corresponding position in
+	 * the boolean array is {@code true}.
+	 * The number of fields in the result is consequently equal to the number of times that {@code true}
+	 * occurs in the fields array.
+	 *
+	 * @param vertexFields The array of flags that describes which fields are to be included from the CSV file for vertices.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsVertices(boolean ... vertexFields) {
+		if(this.vertexReader != null) {
+			this.vertexReader.includeFields(vertexFields);
+		}
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
+	 * parser will look at the first {@code n} fields, where {@code n} is the length of the boolean
+	 * array. The parser will skip over all fields where the boolean value at the corresponding position
+	 * in the array is {@code false}. The result contains the fields where the corresponding position in
+	 * the boolean array is {@code true}.
+	 * The number of fields in the result is consequently equal to the number of times that {@code true}
+	 * occurs in the fields array.
+	 *
+	 * @param edgeFields The array of flags that describes which fields are to be included from the CSV file for edges.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsEdges(boolean ... edgeFields) {
+		this.edgeReader.includeFields(edgeFields);
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
+	 * positions in the string (read from position 0 to its length) define whether the field at
+	 * the corresponding position in the CSV schema should be included.
+	 * parser will look at the first {@code n} fields, where {@code n} is the length of the mask string
+	 * The parser will skip over all fields where the character at the corresponding position
+	 * in the string is {@code '0'}, {@code 'F'}, or {@code 'f'} (representing the value
+	 * {@code false}). The result contains the fields where the corresponding position in
+	 * the boolean array is {@code '1'}, {@code 'T'}, or {@code 't'} (representing the value {@code true}).
+	 *
+	 * @param mask The string mask defining which fields to include and which to skip.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsVertices(String mask) {
+		if(this.vertexReader != null) {
+			this.vertexReader.includeFields(mask);
+		}
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
+	 * positions in the string (read from position 0 to its length) define whether the field at
+	 * the corresponding position in the CSV schema should be included.
+	 * parser will look at the first {@code n} fields, where {@code n} is the length of the mask string
+	 * The parser will skip over all fields where the character at the corresponding position
+	 * in the string is {@code '0'}, {@code 'F'}, or {@code 'f'} (representing the value
+	 * {@code false}). The result contains the fields where the corresponding position in
+	 * the boolean array is {@code '1'}, {@code 'T'}, or {@code 't'} (representing the value {@code true}).
+	 *
+	 * @param mask The string mask defining which fields to include and which to skip.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsEdges(String mask) {
+		this.edgeReader.includeFields(mask);
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
+	 * bits in the value (read from least significant to most significant) define whether the field at
+	 * the corresponding position in the CSV schema should be included.
+	 * parser will look at the first {@code n} fields, where {@code n} is the position of the most significant
+	 * non-zero bit.
+	 * The parser will skip over all fields where the character at the corresponding bit is zero, and
+	 * include the fields where the corresponding bit is one.
+	 * <p>
+	 * Examples:
+	 * <ul>
+	 *   <li>A mask of {@code 0x7} would include the first three fields.</li>
+	 *   <li>A mask of {@code 0x26} (binary {@code 100110} would skip the first fields, include fields
+	 *       two and three, skip fields four and five, and include field six.</li>
+	 * </ul>
+	 *
+	 * @param mask The bit mask defining which fields to include and which to skip.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsVertices(long mask) {
+		if(this.vertexReader != null) {
+			this.vertexReader.includeFields(mask);
+		}
+		return this;
+	}
+
+	/**
+	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
+	 * bits in the value (read from least significant to most significant) define whether the field at
+	 * the corresponding position in the CSV schema should be included.
+	 * parser will look at the first {@code n} fields, where {@code n} is the position of the most significant
+	 * non-zero bit.
+	 * The parser will skip over all fields where the character at the corresponding bit is zero, and
+	 * include the fields where the corresponding bit is one.
+	 * <p>
+	 * Examples:
+	 * <ul>
+	 *   <li>A mask of {@code 0x7} would include the first three fields.</li>
+	 *   <li>A mask of {@code 0x26} (binary {@code 100110} would skip the first fields, include fields
+	 *       two and three, skip fields four and five, and include field six.</li>
+	 * </ul>
+	 *
+	 * @param mask The bit mask defining which fields to include and which to skip.
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader includeFieldsEdges(long mask) {
+		this.edgeReader.includeFields(mask);
+		return this;
+	}
+
+	/**
+	 * Sets the CSV reader for the Edges file to ignore the first line. This is useful for files that contain a header line.
+	 *
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreFirstLineEdges() {
+		this.edgeReader.ignoreFirstLine();
+		return this;
+	}
+
+	/**
+	 * Sets the CSV reader for the Vertices file to ignore the first line. This is useful for files that contain a header line.
+	 *
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreFirstLineVertices() {
+		if(this.vertexReader != null) {
+			this.vertexReader.ignoreFirstLine();
+		}
+		return this;
+	}
+
+	/**
+	 * Sets the CSV reader for the Edges file  to ignore any invalid lines.
+	 * This is useful for files that contain an empty line at the end, multiple header lines or comments. This would throw an exception otherwise.
+	 *
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreInvalidLinesEdges() {
+		this.edgeReader.ignoreInvalidLines();
+		return this;
+	}
+
+	/**
+	 * Sets the CSV reader Vertices file  to ignore any invalid lines.
+	 * This is useful for files that contain an empty line at the end, multiple header lines or comments. This would throw an exception otherwise.
+	 *
+	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
+	 */
+	public GraphCsvReader ignoreInvalidLinesVertices() {
+		if(this.vertexReader != null) {
+			this.vertexReader.ignoreInvalidLines();
+		}
+		return this;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
new file mode 100644
index 0000000..3215194
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import com.google.common.base.Preconditions;
+
+/**
+ * This is used as a base class for vertex-centric iteration or gather-sum-apply iteration configuration.
+ */
+public abstract class IterationConfiguration {
+
+	/** the iteration name **/
+	private String name;
+
+	/** the iteration parallelism **/
+	private int parallelism = -1;
+
+	/** the iteration aggregators **/
+	private Map<String, Aggregator<?>> aggregators = new HashMap<String, Aggregator<?>>();
+
+	/** flag that defines whether the solution set is kept in managed memory **/
+	private boolean unmanagedSolutionSet = false;
+
+	/** flag that defines whether the number of vertices option is set **/
+	private boolean optNumVertices = false;
+	
+	public IterationConfiguration() {}
+
+	/**
+	 * Sets the name for the iteration. The name is displayed in logs and messages.
+	 * 
+	 * @param name The name for the iteration.
+	 */
+	public void setName(String name) {
+		this.name = name;
+	}
+
+	/**
+	 * Gets the name of the iteration.
+	 * @param defaultName 
+	 * 
+	 * @return The name of the iteration.
+	 */
+	public String getName(String defaultName) {
+		if (name != null) {
+			return name;			
+		}
+		else {
+			return defaultName;
+		}
+	}
+
+	/**
+	 * Sets the parallelism for the iteration.
+	 * 
+	 * @param parallelism The parallelism.
+	 */
+	public void setParallelism(int parallelism) {
+		Preconditions.checkArgument(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default).");
+		this.parallelism = parallelism;
+	}
+	
+	/**
+	 * Gets the iteration's parallelism.
+	 * 
+	 * @return The iterations parallelism, or -1, if not set.
+	 */
+	public int getParallelism() {
+		return parallelism;
+	}
+
+	/**
+	 * Defines whether the solution set is kept in managed memory (Flink's internal way of keeping object
+	 * in serialized form) or as a simple object map.
+	 * By default, the solution set runs in managed memory.
+	 * 
+	 * @param unmanaged True, to keep the solution set in unmanaged memory, false otherwise.
+	 */
+	public void setSolutionSetUnmanagedMemory(boolean unmanaged) {
+		this.unmanagedSolutionSet = unmanaged;
+	}
+	
+	/**
+	 * Gets whether the solution set is kept in managed memory (Flink's internal way of keeping object
+	 * in serialized form) or as a simple object map.
+	 * By default, the solution set runs in managed memory.
+	 * 
+	 * @return True, if the solution set is in unmanaged memory, false otherwise.
+	 */
+	public boolean isSolutionSetUnmanagedMemory() {
+		return this.unmanagedSolutionSet;
+	}
+
+	/**
+	 * Gets whether the number of vertices option is set.
+	 * By default, the number of vertices option is not set.
+	 *
+	 * @return True, if the number of vertices option is set, false otherwise.
+	 */
+	public boolean isOptNumVertices() {
+		return optNumVertices;
+	}
+
+	/**
+	 * Sets the number of vertices option.
+	 * By default, the number of vertices option is not set.
+	 *
+	 * @param optNumVertices True, to set this option, false otherwise.
+	 */
+	public void setOptNumVertices(boolean optNumVertices) {
+		this.optNumVertices = optNumVertices;
+	}
+
+	/**
+	 * Registers a new aggregator. Aggregators registered here are available during the execution of the vertex updates
+	 * via {@link org.apache.flink.graph.spargel.VertexUpdateFunction#getIterationAggregator(String)} and
+	 * {@link org.apache.flink.graph.spargel.VertexUpdateFunction#getPreviousIterationAggregate(String)}.
+	 * 
+	 * @param name The name of the aggregator, used to retrieve it and its aggregates during execution. 
+	 * @param aggregator The aggregator.
+	 */
+	public void registerAggregator(String name, Aggregator<?> aggregator) {
+		this.aggregators.put(name, aggregator);
+	}
+
+	/**
+	 * Gets the set of aggregators that are registered for this vertex-centric iteration.
+	 *
+	 * @return a Map of the registered aggregators, where the key is the aggregator name
+	 * and the value is the Aggregator object
+	 */
+	public Map<String, Aggregator<?>> getAggregators() {
+		return this.aggregators;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
new file mode 100644
index 0000000..a21b23d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.util.Collector;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#groupReduceOnNeighbors(NeighborsFunction, EdgeDirection)}
+ * method.
+ *
+ * @param <K> the vertex key type
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ * @param <O> the type of the return value
+ */
+public interface NeighborsFunction<K, VV, EV, O> extends Function, Serializable {
+
+	void iterateNeighbors(Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
new file mode 100644
index 0000000..fdf54fa
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#groupReduceOnNeighbors(NeighborsFunctionWithVertexValue, EdgeDirection)}
+ * method.
+ *
+ * @param <K> the vertex key type
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ * @param <O> the type of the return value
+ */
+public interface NeighborsFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
+
+	void iterateNeighbors(Vertex<K, VV> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
new file mode 100644
index 0000000..84eec51
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.graph;
+
+import org.apache.flink.api.common.functions.Function;
+
+import java.io.Serializable;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#reduceOnEdges(org.apache.flink.graph.ReduceEdgesFunction, EdgeDirection)} method.
+ *
+ * @param <EV> the edge value type
+ */
+public interface ReduceEdgesFunction<EV> extends Function, Serializable {
+
+	EV reduceEdges(EV firstEdgeValue, EV secondEdgeValue);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
new file mode 100644
index 0000000..fc5295d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.functions.Function;
+
+import java.io.Serializable;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#reduceOnNeighbors(ReduceNeighborsFunction, EdgeDirection)}
+ * method.
+ *
+ * @param <VV> the vertex value type
+ */
+public interface ReduceNeighborsFunction <VV> extends Function, Serializable {
+
+	VV reduceNeighbors(VV firstNeighborValue, VV secondNeighborValue);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
new file mode 100644
index 0000000..dee3480
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+import org.apache.flink.api.java.tuple.Tuple5;
+
+/**
+ * A Triplet stores and retrieves the edges along with their corresponding source and target vertices.
+ * Triplets can be obtained from the input graph via the {@link org.apache.flink.graph.Graph#getTriplets()} method.
+ *
+ * @param <K> the vertex key type
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ */
+public class Triplet <K, VV, EV> extends Tuple5<K, K, VV, VV, EV> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Triplet() {}
+
+	/**
+	 * Constructs a Triplet from a given source vertex, target vertex and edge
+	 *
+	 * @param srcVertex
+	 * @param trgVertex
+	 * @param edge
+	 */
+	public Triplet(Vertex<K, VV> srcVertex, Vertex<K, VV> trgVertex, Edge<K, EV> edge) {
+		this.f0 = srcVertex.f0;
+		this.f2 = srcVertex.f1;
+		this.f1 = trgVertex.f0;
+		this.f3 = trgVertex.f1;
+		this.f4 = edge.f2;
+	}
+
+	/**
+	 * Constructs a Triplet from its src vertex id, src target id, src vertex value,
+	 * src target value and edge value respectively.
+	 *
+	 * @param srcId
+	 * @param trgId
+	 * @param srcVal
+	 * @param trgVal
+	 * @param edgeVal
+	 */
+	public Triplet(K srcId, K trgId, VV srcVal, VV trgVal, EV edgeVal) {
+		super(srcId, trgId, srcVal, trgVal, edgeVal);
+	}
+
+	public Vertex<K, VV> getSrcVertex() {
+		return new Vertex<K, VV>(this.f0, this.f2);
+	}
+
+	public Vertex<K, VV> getTrgVertex() {
+		return new Vertex<K, VV>(this.f1, this.f3);
+	}
+
+	public Edge<K, EV> getEdge() {
+		return new Edge<K, EV>(this.f0, this.f1, this.f4);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java
new file mode 100644
index 0000000..c5eb973
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.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;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/**
+ * Represents the graph's nodes. It carries an ID and a value.
+ * For vertices with no value, use {@link org.apache.flink.types.NullValue} as the value type.
+ *
+ * @param <K>
+ * @param <V>
+ */
+public class Vertex<K, V> extends Tuple2<K, V> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Vertex(){}
+
+	public Vertex(K k, V val) {
+		this.f0 = k;
+		this.f1 = val;
+	}
+
+	public K getId() {
+		return this.f0;
+	}
+
+	public V getValue() {
+		return this.f1;
+	}
+
+	public void setId(K id) {
+		this.f0 = id;
+	}
+
+	public void setValue(V val) {
+		this.f1 = val;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java
new file mode 100644
index 0000000..cd52e04
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java
@@ -0,0 +1,141 @@
+/*
+ * 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.example;
+
+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.example.utils.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 org.apache.flink.graph.library.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 org.apache.flink.graph.example.utils.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, 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<Long, NullValue>(value.f0, value.f1, NullValue.getInstance());
+						}
+					});
+		} else {
+			return ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
new file mode 100644
index 0000000..b7e3385
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
@@ -0,0 +1,215 @@
+/*
+ * 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.example;
+
+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.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Triplet;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.EuclideanGraphData;
+
+import java.io.Serializable;
+
+/**
+ * This example shows how to use Gelly's {@link Graph#getTriplets()} and
+ * {@link Graph#joinWithEdges(DataSet, MapFunction)} methods.
+ * 
+ * Given a directed, unweighted graph, with vertex values representing points in a plan,
+ * return a weighted graph where the edge weights are equal to the Euclidean distance between the
+ * src and the trg vertex values.
+ *
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <ul>
+ * 	<li> Vertices are represented by their vertexIds and vertex values and are separated by newlines,
+ * 	the value being formed of two doubles separated by a comma.
+ * 	For example: <code>1,1.0,1.0\n2,2.0,2.0\n3,3.0,3.0\n</code> defines a data set of three vertices
+ * 	<li> Edges are represented by pairs of srcVertexId, trgVertexId separated by commas.
+ * 	Edges themselves are separated by newlines.
+ * 	For example: <code>1,2\n1,3\n</code> defines two edges 1-2 and 1-3.
+ * </ul>
+ * </p>
+ *
+ * Usage <code>EuclideanGraphWeighing &lt;vertex path&gt; &lt;edge path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.graph.example.utils.EuclideanGraphData}
+ */
+@SuppressWarnings("serial")
+public class EuclideanGraphWeighing implements ProgramDescription {
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Vertex<Long, Point>> vertices = getVerticesDataSet(env);
+
+		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
+
+		Graph<Long, Point, Double> graph = Graph.fromDataSet(vertices, edges, env);
+
+		// the edge value will be the Euclidean distance between its src and trg vertex
+		DataSet<Tuple3<Long, Long, Double>> edgesWithEuclideanWeight = graph.getTriplets()
+				.map(new MapFunction<Triplet<Long, Point, Double>, Tuple3<Long, Long, Double>>() {
+
+					@Override
+					public Tuple3<Long, Long, Double> map(Triplet<Long, Point, Double> triplet)
+							throws Exception {
+
+						Vertex<Long, Point> srcVertex = triplet.getSrcVertex();
+						Vertex<Long, Point> trgVertex = triplet.getTrgVertex();
+
+						return new Tuple3<Long, Long, Double>(srcVertex.getId(), trgVertex.getId(),
+								srcVertex.getValue().euclideanDistance(trgVertex.getValue()));
+					}
+				});
+
+		Graph<Long, Point, Double> resultedGraph = graph.joinWithEdges(edgesWithEuclideanWeight,
+				new MapFunction<Tuple2<Double, Double>, Double>() {
+
+					@Override
+					public Double map(Tuple2<Double, Double> distance) throws Exception {
+						return distance.f1;
+					}
+				});
+
+		// retrieve the edges from the final result
+		DataSet<Edge<Long, Double>> result = resultedGraph.getEdges();
+
+		// emit result
+		if (fileOutput) {
+			result.writeAsCsv(outputPath, "\n", ",");
+
+			// since file sinks are lazy, we trigger the execution explicitly
+			env.execute("Euclidean Graph Weighing Example");
+		} else {
+			result.print();
+		}
+
+	}
+
+	@Override
+	public String getDescription() {
+		return "Weighing a graph by computing the Euclidean distance " +
+				"between its vertices";
+	}
+
+	// *************************************************************************
+	//     DATA TYPES
+	// *************************************************************************
+
+	/**
+	 * A simple two-dimensional point.
+	 */
+	public static class Point implements Serializable {
+
+		public double x, y;
+
+		public Point() {}
+
+		public Point(double x, double y) {
+			this.x = x;
+			this.y = y;
+		}
+
+		public double euclideanDistance(Point other) {
+			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
+		}
+
+		@Override
+		public String toString() {
+			return x + " " + y;
+		}
+	}
+
+	// ******************************************************************************************************************
+	// UTIL METHODS
+	// ******************************************************************************************************************
+
+	private static boolean fileOutput = false;
+
+	private static String verticesInputPath = null;
+
+	private static String edgesInputPath = null;
+
+	private static String outputPath = null;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			if (args.length == 3) {
+				fileOutput = true;
+				verticesInputPath = args[0];
+				edgesInputPath = args[1];
+				outputPath = args[2];
+			} else {
+				System.out.println("Executing Euclidean Graph Weighing example with default parameters and built-in default data.");
+				System.out.println("Provide parameters to read input data from files.");
+				System.out.println("See the documentation for the correct format of input files.");
+				System.err.println("Usage: EuclideanGraphWeighing <input vertices path> <input edges path>" +
+						" <output path>");
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static DataSet<Vertex<Long, Point>> getVerticesDataSet(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(verticesInputPath)
+					.lineDelimiter("\n")
+					.types(Long.class, Double.class, Double.class)
+					.map(new MapFunction<Tuple3<Long, Double, Double>, Vertex<Long, Point>>() {
+
+						@Override
+						public Vertex<Long, Point> map(Tuple3<Long, Double, Double> value) throws Exception {
+							return new Vertex<Long, Point>(value.f0, new Point(value.f1, value.f2));
+						}
+					});
+		} else {
+			return EuclideanGraphData.getDefaultVertexDataSet(env);
+		}
+	}
+
+	private static DataSet<Edge<Long, Double>> getEdgesDataSet(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(edgesInputPath)
+					.lineDelimiter("\n")
+					.types(Long.class, Long.class)
+					.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, Double>>() {
+
+						@Override
+						public Edge<Long, Double> map(Tuple2<Long, Long> tuple2) throws Exception {
+							return new Edge<Long, Double>(tuple2.f0, tuple2.f1, 0.0);
+						}
+					});
+		} else {
+			return EuclideanGraphData.getDefaultEdgeDataSet(env);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
new file mode 100755
index 0000000..635a099
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
@@ -0,0 +1,191 @@
+/*
+ * 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.example;
+
+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.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
+import org.apache.flink.graph.gsa.ApplyFunction;
+import org.apache.flink.graph.gsa.GatherFunction;
+import org.apache.flink.graph.gsa.SumFunction;
+import org.apache.flink.graph.gsa.Neighbor;
+import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
+
+/**
+ * This example shows how to use Gelly's Gather-Sum-Apply iterations.
+ * 
+ * It is an implementation of the Single-Source-Shortest-Paths algorithm.
+ * For a vertex-centric implementation of the same algorithm, please refer to {@link SingleSourceShortestPaths}. 
+ *
+ * The input file is a plain text file and must be formatted as follows:
+ * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
+ * separated by tabs. Edges themselves are separated by newlines.
+ * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
+ * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
+ *
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData}
+ */
+public class GSASingleSourceShortestPaths implements ProgramDescription {
+
+	// --------------------------------------------------------------------------------------------
+	//  Program
+	// --------------------------------------------------------------------------------------------
+
+	public static void main(String[] args) throws Exception {
+
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Edge<Long, Double>> edges = getEdgeDataSet(env);
+
+		Graph<Long, Double, Double> graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env);
+
+		// Execute the GSA iteration
+		Graph<Long, Double, Double> result = graph.runGatherSumApplyIteration(
+				new CalculateDistances(), new ChooseMinDistance(), new UpdateDistance(), maxIterations);
+
+		// Extract the vertices as the result
+		DataSet<Vertex<Long, Double>> singleSourceShortestPaths = result.getVertices();
+
+		// emit result
+		if(fileOutput) {
+			singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",");
+
+			// since file sinks are lazy, we trigger the execution explicitly
+			env.execute("GSA Single Source Shortest Paths");
+		} else {
+			singleSourceShortestPaths.print();
+		}
+
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Single Source Shortest Path UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static final class InitVertices implements MapFunction<Long, Double>{
+
+		private long srcId;
+
+		public InitVertices(long srcId) {
+			this.srcId = srcId;
+		}
+
+		public Double map(Long id) {
+			if (id.equals(srcId)) {
+				return 0.0;
+			}
+			else {
+				return Double.POSITIVE_INFINITY;
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class CalculateDistances extends GatherFunction<Double, Double, Double> {
+
+		public Double gather(Neighbor<Double, Double> neighbor) {
+			return neighbor.getNeighborValue() + neighbor.getEdgeValue();
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class ChooseMinDistance extends SumFunction<Double, Double, Double> {
+
+		public Double sum(Double newValue, Double currentValue) {
+			return Math.min(newValue, currentValue);
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class UpdateDistance extends ApplyFunction<Long, Double, Double> {
+
+		public void apply(Double newDistance, Double oldDistance) {
+			if (newDistance < oldDistance) {
+				setResult(newDistance);
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Util methods
+	// --------------------------------------------------------------------------------------------
+
+	private static boolean fileOutput = false;
+
+	private static Long srcVertexId = 1l;
+
+	private static String edgesInputPath = null;
+
+	private static String outputPath = null;
+
+	private static int maxIterations = 5;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			if(args.length != 4) {
+				System.err.println("Usage: GSASingleSourceShortestPaths <source vertex id>" +
+						" <input edges path> <output path> <num iterations>");
+				return false;
+			}
+
+			fileOutput = true;
+			srcVertexId = Long.parseLong(args[0]);
+			edgesInputPath = args[1];
+			outputPath = args[2];
+			maxIterations = Integer.parseInt(args[3]);
+		} else {
+				System.out.println("Executing GSASingle Source Shortest Paths example "
+						+ "with default parameters and built-in default data.");
+				System.out.println("  Provide parameters to read input data from files.");
+				System.out.println("  See the documentation for the correct format of input files.");
+				System.out.println("Usage: GSASingleSourceShortestPaths <source vertex id>" +
+						" <input edges path> <output path> <num iterations>");
+		}
+		return true;
+	}
+
+	private static DataSet<Edge<Long, Double>> getEdgeDataSet(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(edgesInputPath)
+					.fieldDelimiter("\t")
+					.lineDelimiter("\n")
+					.types(Long.class, Long.class, Double.class)
+					.map(new Tuple3ToEdgeMap<Long, Double>());
+		} else {
+			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
+		}
+	}
+
+	@Override
+	public String getDescription() {
+		return "GSA Single Source Shortest Paths";
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
new file mode 100644
index 0000000..b808e76
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
@@ -0,0 +1,170 @@
+/*
+ * 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.example;
+
+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.aggregation.Aggregations;
+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.example.utils.ExampleUtils;
+import org.apache.flink.types.NullValue;
+
+/**
+ * This example illustrates how to use Gelly metrics methods and get simple statistics
+ * from the input graph.  
+ * 
+ * The program creates a random graph and computes and prints
+ * the following metrics:
+ * - number of vertices
+ * - number of edges
+ * - average node degree
+ * - the vertex ids with the max/min in- and out-degrees
+ *
+ * The input file is expected to contain one edge per line,
+ * with long IDs and no values, in the following format:
+ * "<sourceVertexID>\t<targetVertexID>".
+ * If no arguments are provided, the example runs with a random graph of 100 vertices.
+ *
+ */
+public class GraphMetrics implements ProgramDescription {
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		/** create the graph **/
+		Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(getEdgesDataSet(env), env);
+		
+		/** get the number of vertices **/
+		long numVertices = graph.numberOfVertices();
+		
+		/** get the number of edges **/
+		long numEdges = graph.numberOfEdges();
+		
+		/** compute the average node degree **/
+		DataSet<Tuple2<Long, Long>> verticesWithDegrees = graph.getDegrees();
+
+		DataSet<Double> avgNodeDegree = verticesWithDegrees
+				.aggregate(Aggregations.SUM, 1).map(new AvgNodeDegreeMapper(numVertices));
+		
+		/** find the vertex with the maximum in-degree **/
+		DataSet<Long> maxInDegreeVertex = graph.inDegrees().maxBy(1).map(new ProjectVertexId());
+
+		/** find the vertex with the minimum in-degree **/
+		DataSet<Long> minInDegreeVertex = graph.inDegrees().minBy(1).map(new ProjectVertexId());
+
+		/** find the vertex with the maximum out-degree **/
+		DataSet<Long> maxOutDegreeVertex = graph.outDegrees().maxBy(1).map(new ProjectVertexId());
+
+		/** find the vertex with the minimum out-degree **/
+		DataSet<Long> minOutDegreeVertex = graph.outDegrees().minBy(1).map(new ProjectVertexId());
+		
+		/** print the results **/
+		ExampleUtils.printResult(env.fromElements(numVertices), "Total number of vertices");
+		ExampleUtils.printResult(env.fromElements(numEdges), "Total number of edges");
+		ExampleUtils.printResult(avgNodeDegree, "Average node degree");
+		ExampleUtils.printResult(maxInDegreeVertex, "Vertex with Max in-degree");
+		ExampleUtils.printResult(minInDegreeVertex, "Vertex with Min in-degree");
+		ExampleUtils.printResult(maxOutDegreeVertex, "Vertex with Max out-degree");
+		ExampleUtils.printResult(minOutDegreeVertex, "Vertex with Min out-degree");
+
+		env.execute();
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AvgNodeDegreeMapper implements MapFunction<Tuple2<Long, Long>, Double> {
+
+		private long numberOfVertices;
+
+		public AvgNodeDegreeMapper(long numberOfVertices) {
+			this.numberOfVertices = numberOfVertices;
+		}
+
+		public Double map(Tuple2<Long, Long> sumTuple) {
+			return (double) (sumTuple.f1 / numberOfVertices) ;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ProjectVertexId implements MapFunction<Tuple2<Long,Long>, Long> {
+		public Long map(Tuple2<Long, Long> value) { return value.f0; }
+	}
+
+	@Override
+	public String getDescription() {
+		return "Graph Metrics Example";
+	}
+
+	// ******************************************************************************************************************
+	// UTIL METHODS
+	// ******************************************************************************************************************
+
+	private static boolean fileOutput = false;
+
+	private static String edgesInputPath = null;
+
+	static final int NUM_VERTICES = 100;
+
+	static final long SEED = 9876;
+
+	private static boolean parseParameters(String[] args) {
+
+		if(args.length > 0) {
+			if(args.length != 1) {
+				System.err.println("Usage: GraphMetrics <input edges>");
+				return false;
+			}
+
+			fileOutput = true;
+			edgesInputPath = args[0];
+		} else {
+			System.out.println("Executing Graph Metrics example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("Usage: GraphMetrics <input edges>");
+		}
+		return true;
+	}
+
+	@SuppressWarnings("serial")
+	private static DataSet<Edge<Long, NullValue>> getEdgesDataSet(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(edgesInputPath)
+					.lineDelimiter("\n").fieldDelimiter("\t")
+					.types(Long.class, Long.class).map(
+							new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
+
+								public Edge<Long, NullValue> map(Tuple2<Long, Long> value) {
+									return new Edge<Long, NullValue>(value.f0, value.f1, 
+											NullValue.getInstance());
+								}
+					});
+		} else {
+			return ExampleUtils.getRandomEdges(env, NUM_VERTICES);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
new file mode 100644
index 0000000..c03937d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
@@ -0,0 +1,268 @@
+/*
+ * 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.example;
+
+import org.apache.flink.api.common.ProgramDescription;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.IncrementalSSSPData;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexCentricConfiguration;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+
+/**
+ * This example illustrates how to 
+ * <ul>
+ *  <li> create a Graph directly from CSV files
+ *  <li> use the vertex-centric iteration's messaging direction configuration option
+ * </ul>
+ * 
+ * Incremental Single Sink Shortest Paths Example. Shortest Paths are incrementally updated
+ * upon edge removal.
+ *
+ * The program takes as input the resulted graph after a SSSP computation,
+ * an edge to be removed and the initial graph(i.e. before SSSP was computed).
+ * In the following description, SP-graph is used as an abbreviation for
+ * the graph resulted from the SSSP computation. We denote the edges that belong to this
+ * graph by SP-edges.
+ *
+ * - If the removed edge does not belong to the SP-graph, no computation is necessary.
+ * The edge is simply removed from the graph.
+ * - If the removed edge is an SP-edge, then all nodes, whose shortest path contains the removed edge,
+ * potentially require re-computation.
+ * When the edge <u, v> is removed, v checks if it has another out-going SP-edge.
+ * If yes, no further computation is required.
+ * If v has no other out-going SP-edge, it invalidates its current value, by setting it to INF.
+ * Then, it informs all its SP-in-neighbors by sending them an INVALIDATE message.
+ * When a vertex u receives an INVALIDATE message from v, it checks whether it has another out-going SP-edge.
+ * If not, it invalidates its current value and propagates the INVALIDATE message.
+ * The propagation stops when a vertex with an alternative shortest path is reached
+ * or when we reach a vertex with no SP-in-neighbors.
+ *
+ * Usage <code>IncrementalSSSP &lt;vertex path&gt; &lt;edge path&gt; &lt;edges in SSSP&gt;
+ * &lt;src id edge to be removed&gt; &lt;trg id edge to be removed&gt; &lt;val edge to be removed&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 org.apache.flink.graph.example.utils.IncrementalSSSPData}
+ */
+@SuppressWarnings("serial")
+public class IncrementalSSSP implements ProgramDescription {
+
+	public static void main(String [] args) throws Exception {
+
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Edge<Long, Double> edgeToBeRemoved = getEdgeToBeRemoved();
+
+		Graph<Long, Double, Double> graph = IncrementalSSSP.getGraph(env);
+
+		// Assumption: all minimum weight paths are kept
+		Graph<Long, Double, Double> ssspGraph = IncrementalSSSP.getSSSPGraph(env);
+
+		// remove the edge
+		graph.removeEdge(edgeToBeRemoved);
+
+		// configure the iteration
+		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+
+		if(isInSSSP(edgeToBeRemoved, ssspGraph.getEdges())) {
+
+			parameters.setDirection(EdgeDirection.IN);
+			parameters.setOptDegrees(true);
+
+			// run the vertex centric iteration to propagate info
+			Graph<Long, Double, Double> result = ssspGraph.runVertexCentricIteration(new VertexDistanceUpdater(),
+					new InvalidateMessenger(edgeToBeRemoved), maxIterations, parameters);
+
+			DataSet<Vertex<Long, Double>> resultedVertices = result.getVertices();
+
+			// Emit results
+			if(fileOutput) {
+				resultedVertices.writeAsCsv(outputPath, "\n", ",");
+				env.execute("Incremental SSSP Example");
+			} else {
+				resultedVertices.print();
+			}
+		} else {
+			// print the vertices
+			if(fileOutput) {
+				graph.getVertices().writeAsCsv(outputPath, "\n", ",");
+				env.execute("Incremental SSSP Example");
+			} else {
+				graph.getVertices().print();
+			}
+		}
+	}
+
+	@Override
+	public String getDescription() {
+		return "Incremental Single Sink Shortest Paths Example";
+	}
+
+	// ******************************************************************************************************************
+	// IncrementalSSSP METHODS
+	// ******************************************************************************************************************
+
+	/**
+	 * Function that verifies whether the edge to be removed is part of the SSSP or not.
+	 * If it is, the src vertex will be invalidated.
+	 *
+	 * @param edgeToBeRemoved
+	 * @param edgesInSSSP
+	 * @return
+	 */
+	public static boolean isInSSSP(final Edge<Long, Double> edgeToBeRemoved, DataSet<Edge<Long, Double>> edgesInSSSP) throws Exception {
+
+		return edgesInSSSP.filter(new FilterFunction<Edge<Long, Double>>() {
+			@Override
+			public boolean filter(Edge<Long, Double> edge) throws Exception {
+				return edge.equals(edgeToBeRemoved);
+			}
+		}).count() > 0;
+	}
+
+	public static final class VertexDistanceUpdater extends VertexUpdateFunction<Long, Double, Double> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Double> vertex, MessageIterator<Double> inMessages) throws Exception {
+			if (inMessages.hasNext()) {
+				Long outDegree = getOutDegree() - 1;
+				// check if the vertex has another SP-Edge
+				if (outDegree > 0) {
+					// there is another shortest path from the source to this vertex
+				} else {
+					// set own value to infinity
+					setNewVertexValue(Double.MAX_VALUE);
+				}
+			}
+		}
+	}
+
+	public static final class InvalidateMessenger extends MessagingFunction<Long, Double, Double, Double> {
+
+		private Edge<Long, Double> edgeToBeRemoved;
+
+		public InvalidateMessenger(Edge<Long, Double> edgeToBeRemoved) {
+			this.edgeToBeRemoved = edgeToBeRemoved;
+		}
+
+		@Override
+		public void sendMessages(Vertex<Long, Double> vertex) throws Exception {
+
+
+			if(getSuperstepNumber() == 1) {
+				if(vertex.getId().equals(edgeToBeRemoved.getSource())) {
+					// activate the edge target
+					sendMessageTo(edgeToBeRemoved.getSource(), Double.MAX_VALUE);
+				}
+			}
+
+			if(getSuperstepNumber() > 1) {
+				// invalidate all edges
+				for(Edge<Long, Double> edge : getEdges()) {
+					sendMessageTo(edge.getSource(), Double.MAX_VALUE);
+				}
+			}
+		}
+	}
+
+	// ******************************************************************************************************************
+	// UTIL METHODS
+	// ******************************************************************************************************************
+
+	private static boolean fileOutput = false;
+
+	private static String verticesInputPath = null;
+
+	private static String edgesInputPath = null;
+
+	private static String edgesInSSSPInputPath = null;
+
+	private static Long srcEdgeToBeRemoved = null;
+
+	private static Long trgEdgeToBeRemoved = null;
+
+	private static Double valEdgeToBeRemoved = null;
+
+	private static String outputPath = null;
+
+	private static int maxIterations = 5;
+
+	private static boolean parseParameters(String[] args) {
+		if (args.length > 0) {
+			if (args.length == 8) {
+				fileOutput = true;
+				verticesInputPath = args[0];
+				edgesInputPath = args[1];
+				edgesInSSSPInputPath = args[2];
+				srcEdgeToBeRemoved = Long.parseLong(args[3]);
+				trgEdgeToBeRemoved = Long.parseLong(args[4]);
+				valEdgeToBeRemoved = Double.parseDouble(args[5]);
+				outputPath = args[6];
+				maxIterations = Integer.parseInt(args[7]);
+			} else {
+				System.out.println("Executing IncrementalSSSP example with default parameters and built-in default data.");
+				System.out.println("Provide parameters to read input data from files.");
+				System.out.println("See the documentation for the correct format of input files.");
+				System.out.println("Usage: IncrementalSSSP <vertex path> <edge path> <edges in SSSP> " +
+						"<src id edge to be removed> <trg id edge to be removed> <val edge to be removed> " +
+						"<output path> <max iterations>");
+
+				return false;
+			}
+		}
+		return true;
+	}
+
+	private static Graph<Long, Double, Double> getGraph(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return Graph.fromCsvReader(verticesInputPath, edgesInputPath, env).lineDelimiterEdges("\n")
+					.types(Long.class, Double.class, Double.class);
+		} else {
+			return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgeDataSet(env), env);
+		}
+	}
+
+	private static Graph<Long, Double, Double> getSSSPGraph(ExecutionEnvironment env) {
+		if(fileOutput) {
+			return Graph.fromCsvReader(verticesInputPath, edgesInSSSPInputPath, env).lineDelimiterEdges("\n")
+					.types(Long.class, Double.class, Double.class);
+		} else {
+			return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgesInSSSP(env), env);
+		}
+	}
+
+	private static Edge<Long, Double> getEdgeToBeRemoved() {
+		if (fileOutput) {
+			return new Edge<Long, Double>(srcEdgeToBeRemoved, trgEdgeToBeRemoved, valEdgeToBeRemoved);
+		} else {
+			return IncrementalSSSPData.getDefaultEdgeToBeRemoved();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
new file mode 100644
index 0000000..0f84dbb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
@@ -0,0 +1,214 @@
+/*
+ * 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.example;
+
+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.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.ReduceNeighborsFunction;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.Triplet;
+import org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData;
+
+import java.util.HashSet;
+
+/**
+ * This example shows how to use
+ * <ul>
+ *  <li> neighborhood methods
+ *  <li> join with vertices
+ *  <li> triplets
+ * </ul>
+ * 
+ * Given a directed, unweighted graph, return a weighted graph where the edge values are equal
+ * to the Jaccard similarity coefficient - the number of common neighbors divided by the the size
+ * of the union of neighbor sets - for the src and target vertices.
+ *
+ * <p>
+ * Input files are plain text files and must be formatted as follows:
+ * <br>
+ * 	Edges are represented by pairs of srcVertexId, trgVertexId separated by tabs.
+ * 	Edges themselves are separated by newlines.
+ * 	For example: <code>1	2\n1	3\n</code> defines two edges 1-2 and 1-3.
+ * </p>
+ *
+ * Usage <code> JaccardSimilarityMeasure &lt;edge path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData}
+ */
+@SuppressWarnings("serial")
+public class JaccardSimilarityMeasure implements ProgramDescription {
+
+	public static void main(String [] args) throws Exception {
+
+		if(!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
+
+		Graph<Long, HashSet<Long>, Double> graph = Graph.fromDataSet(edges,
+				new MapFunction<Long, HashSet<Long>>() {
+
+					@Override
+					public HashSet<Long> map(Long id) throws Exception {
+						HashSet<Long> neighbors = new HashSet<Long>();
+						neighbors.add(id);
+
+						return new HashSet<Long>(neighbors);
+					}
+				}, env);
+
+		// create the set of neighbors
+		DataSet<Tuple2<Long, HashSet<Long>>> computedNeighbors =
+				graph.reduceOnNeighbors(new GatherNeighbors(), EdgeDirection.ALL);
+
+		// join with the vertices to update the node values
+		Graph<Long, HashSet<Long>, Double> graphWithVertexValues =
+				graph.joinWithVertices(computedNeighbors, new MapFunction<Tuple2<HashSet<Long>, HashSet<Long>>,
+						HashSet<Long>>() {
+
+					@Override
+					public HashSet<Long> map(Tuple2<HashSet<Long>, HashSet<Long>> tuple2) throws Exception {
+						return tuple2.f1;
+					}
+				});
+
+		// compare neighbors, compute Jaccard
+		DataSet<Edge<Long, Double>> edgesWithJaccardValues =
+				graphWithVertexValues.getTriplets().map(new ComputeJaccard());
+
+		// emit result
+		if (fileOutput) {
+			edgesWithJaccardValues.writeAsCsv(outputPath, "\n", ",");
+
+			// since file sinks are lazy, we trigger the execution explicitly
+			env.execute("Executing Jaccard Similarity Measure");
+		} else {
+			edgesWithJaccardValues.print();
+		}
+
+	}
+
+	@Override
+	public String getDescription() {
+		return "Vertex Jaccard Similarity Measure";
+	}
+
+	/**
+	 * Each vertex will have a HashSet containing its neighbor ids as value.
+	 */
+	private static final class GatherNeighbors implements ReduceNeighborsFunction<HashSet<Long>> {
+
+		@Override
+		public HashSet<Long> reduceNeighbors(HashSet<Long> first, HashSet<Long> second) {
+			first.addAll(second);
+			return new HashSet<Long>(first);
+		}
+	}
+
+	/**
+	 * The edge weight will be the Jaccard coefficient, which is computed as follows:
+	 *
+	 * Consider the edge x-y
+	 * We denote by sizeX and sizeY, the neighbors hash set size of x and y respectively.
+	 * sizeX+sizeY = union + intersection of neighborhoods
+	 * size(hashSetX.addAll(hashSetY)).distinct = union of neighborhoods
+	 * The intersection can then be deduced.
+	 *
+	 * The Jaccard similarity coefficient is then, the intersection/union.
+	 */
+	private static final class ComputeJaccard implements
+			MapFunction<Triplet<Long, HashSet<Long>, Double>, Edge<Long, Double>> {
+
+		@Override
+		public Edge<Long, Double> map(Triplet<Long, HashSet<Long>, Double> triplet) throws Exception {
+
+			Vertex<Long, HashSet<Long>> srcVertex = triplet.getSrcVertex();
+			Vertex<Long, HashSet<Long>> trgVertex = triplet.getTrgVertex();
+
+			Long x = srcVertex.getId();
+			Long y = trgVertex.getId();
+			HashSet<Long> neighborSetY = trgVertex.getValue();
+
+			double unionPlusIntersection = srcVertex.getValue().size() + neighborSetY.size();
+			// within a HashSet, all elements are distinct
+			HashSet<Long> unionSet = new HashSet<Long>();
+			unionSet.addAll(srcVertex.getValue());
+			unionSet.addAll(neighborSetY);
+			double union = unionSet.size();
+			double intersection = unionPlusIntersection - union;
+
+			return new Edge<Long, Double>(x, y, intersection/union);
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String edgeInputPath = null;
+	private static String outputPath = null;
+
+	private static boolean parseParameters(String [] args) {
+		if(args.length > 0) {
+			if(args.length != 2) {
+				System.err.println("Usage JaccardSimilarityMeasure <edge path> <output path>");
+				return false;
+			}
+
+			fileOutput = true;
+			edgeInputPath = args[0];
+			outputPath = args[1];
+		} else {
+			System.out.println("Executing JaccardSimilarityMeasure example with default parameters and built-in default data.");
+			System.out.println("Provide parameters to read input data from files.");
+			System.out.println("Usage JaccardSimilarityMeasure <edge path> <output path>");
+		}
+
+		return true;
+	}
+
+	private static DataSet<Edge<Long, Double>> 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, Double>>() {
+						@Override
+						public Edge<Long, Double> map(Tuple2<Long, Long> tuple2) throws Exception {
+							return new Edge<Long, Double>(tuple2.f0, tuple2.f1, new Double(0));
+						}
+					});
+		} else {
+			return JaccardSimilarityMeasureData.getDefaultEdgeDataSet(env);
+		}
+	}
+}


[21/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/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
new file mode 100755
index 0000000..b24f749
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
@@ -0,0 +1,1948 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.List;
+import java.util.Arrays;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+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.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.graph.gsa.ApplyFunction;
+import org.apache.flink.graph.gsa.GSAConfiguration;
+import org.apache.flink.graph.gsa.GatherFunction;
+import org.apache.flink.graph.gsa.GatherSumApplyIteration;
+import org.apache.flink.graph.gsa.SumFunction;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexCentricConfiguration;
+import org.apache.flink.graph.spargel.VertexCentricIteration;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.graph.utils.EdgeToTuple3Map;
+import org.apache.flink.graph.utils.Tuple2ToVertexMap;
+import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
+import org.apache.flink.graph.utils.VertexToTuple2Map;
+import org.apache.flink.graph.validation.GraphValidator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.types.NullValue;
+
+/**
+ * Represents a Graph consisting of {@link Edge edges} and {@link Vertex
+ * vertices}.
+ * 
+ * 
+ * @see org.apache.flink.graph.Edge
+ * @see org.apache.flink.graph.Vertex
+ * 
+ * @param <K> the key type for edge and vertex identifiers
+ * @param <VV> the value type for vertices
+ * @param <EV> the value type for edges
+ */
+@SuppressWarnings("serial")
+public class Graph<K, VV, EV> {
+
+	private final ExecutionEnvironment context;
+	private final DataSet<Vertex<K, VV>> vertices;
+	private final DataSet<Edge<K, EV>> edges;
+
+	/**
+	 * Creates a graph from two DataSets: vertices and edges
+	 * 
+	 * @param vertices a DataSet of vertices.
+	 * @param edges a DataSet of edges.
+	 * @param context the flink execution environment.
+	 */
+	private Graph(DataSet<Vertex<K, VV>> vertices, DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
+		this.vertices = vertices;
+		this.edges = edges;
+		this.context = context;
+	}
+
+	/**
+	 * Creates a graph from a Collection of vertices and a Collection of edges.
+	 * 
+	 * @param vertices a Collection of vertices.
+	 * @param edges a Collection of edges.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Vertex<K, VV>> vertices,
+			Collection<Edge<K, EV>> edges, ExecutionEnvironment context) {
+
+		return fromDataSet(context.fromCollection(vertices),
+				context.fromCollection(edges), context);
+	}
+
+	/**
+	 * Creates a graph from a Collection of edges, vertices are induced from the
+	 * edges. Vertices are created automatically and their values are set to
+	 * NullValue.
+	 * 
+	 * @param edges a Collection of vertices.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, EV> Graph<K, NullValue, EV> fromCollection(Collection<Edge<K, EV>> edges,
+			ExecutionEnvironment context) {
+
+		return fromDataSet(context.fromCollection(edges), context);
+	}
+
+	/**
+	 * Creates a graph from a Collection of edges, vertices are induced from the
+	 * edges and vertex values are calculated by a mapper function. Vertices are
+	 * created automatically and their values are set by applying the provided
+	 * map function to the vertex ids.
+	 * 
+	 * @param edges a Collection of edges.
+	 * @param mapper the mapper function.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Edge<K, EV>> edges,
+			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+
+		return fromDataSet(context.fromCollection(edges), mapper, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of vertices and a DataSet of edges.
+	 * 
+	 * @param vertices a DataSet of vertices.
+	 * @param edges a DataSet of edges.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromDataSet(DataSet<Vertex<K, VV>> vertices,
+			DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
+
+		return new Graph<K, VV, EV>(vertices, edges, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of edges, vertices are induced from the
+	 * edges. Vertices are created automatically and their values are set to
+	 * NullValue.
+	 * 
+	 * @param edges a DataSet of edges.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, EV> Graph<K, NullValue, EV> fromDataSet(
+			DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
+
+		DataSet<Vertex<K, NullValue>> vertices = edges.flatMap(new EmitSrcAndTarget<K, EV>()).distinct();
+
+		return new Graph<K, NullValue, EV>(vertices, edges, context);
+	}
+
+	private static final class EmitSrcAndTarget<K, EV> implements FlatMapFunction<
+		Edge<K, EV>, Vertex<K, NullValue>> {
+
+		public void flatMap(Edge<K, EV> edge, Collector<Vertex<K, NullValue>> out) {
+			out.collect(new Vertex<K, NullValue>(edge.f0, NullValue.getInstance()));
+			out.collect(new Vertex<K, NullValue>(edge.f1, NullValue.getInstance()));
+		}
+	}
+
+	/**
+	 * Creates a graph from a DataSet of edges, vertices are induced from the
+	 * edges and vertex values are calculated by a mapper function. Vertices are
+	 * created automatically and their values are set by applying the provided
+	 * map function to the vertex ids.
+	 * 
+	 * @param edges a DataSet of edges.
+	 * @param mapper the mapper function.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromDataSet(DataSet<Edge<K, EV>> edges,
+			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+
+		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
+
+		TypeInformation<VV> valueType = TypeExtractor.createTypeInfo(
+				MapFunction.class, mapper.getClass(), 1, null, null);
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		TypeInformation<Vertex<K, VV>> returnType = (TypeInformation<Vertex<K, VV>>) new TupleTypeInfo(
+				Vertex.class, keyType, valueType);
+
+		DataSet<Vertex<K, VV>> vertices = edges
+				.flatMap(new EmitSrcAndTargetAsTuple1<K, EV>()).distinct()
+				.map(new MapFunction<Tuple1<K>, Vertex<K, VV>>() {
+					public Vertex<K, VV> map(Tuple1<K> value) throws Exception {
+						return new Vertex<K, VV>(value.f0, mapper.map(value.f0));
+					}
+				}).returns(returnType).withForwardedFields("f0");
+
+		return new Graph<K, VV, EV>(vertices, edges, context);
+	}
+
+	private static final class EmitSrcAndTargetAsTuple1<K, EV> implements FlatMapFunction<
+		Edge<K, EV>, Tuple1<K>> {
+
+		public void flatMap(Edge<K, EV> edge, Collector<Tuple1<K>> out) {
+			out.collect(new Tuple1<K>(edge.f0));
+			out.collect(new Tuple1<K>(edge.f1));
+		}
+	}
+
+	/**
+	 * Creates a graph from a DataSet of Tuple objects for vertices and edges.
+	 * 
+	 * Vertices with value are created from Tuple2, Edges with value are created
+	 * from Tuple3.
+	 * 
+	 * @param vertices a DataSet of Tuple2.
+	 * @param edges a DataSet of Tuple3.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromTupleDataSet(DataSet<Tuple2<K, VV>> vertices,
+			DataSet<Tuple3<K, K, EV>> edges, ExecutionEnvironment context) {
+
+		DataSet<Vertex<K, VV>> vertexDataSet = vertices.map(new Tuple2ToVertexMap<K, VV>());
+		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
+		return fromDataSet(vertexDataSet, edgeDataSet, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
+	 * induced from the edges.
+	 * 
+	 * Edges with value are created from Tuple3. Vertices are created
+	 * automatically and their values are set to NullValue.
+	 * 
+	 * @param edges a DataSet of Tuple3.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, EV> Graph<K, NullValue, EV> fromTupleDataSet(DataSet<Tuple3<K, K, EV>> edges,
+			ExecutionEnvironment context) {
+
+		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
+		return fromDataSet(edgeDataSet, context);
+	}
+
+	/**
+	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
+	 * induced from the edges and vertex values are calculated by a mapper
+	 * function. Edges with value are created from Tuple3. Vertices are created
+	 * automatically and their values are set by applying the provided map
+	 * function to the vertex ids.
+	 * 
+	 * @param edges a DataSet of Tuple3.
+	 * @param mapper the mapper function.
+	 * @param context the flink execution environment.
+	 * @return the newly created graph.
+	 */
+	public static <K, VV, EV> Graph<K, VV, EV> fromTupleDataSet(DataSet<Tuple3<K, K, EV>> edges,
+			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+
+		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
+		return fromDataSet(edgeDataSet, mapper, context);
+	}
+
+	/**
+	* Creates a Graph from a CSV file of vertices and a CSV file of edges.
+	* 
+	* @param verticesPath path to a CSV file with the Vertex data.
+	* @param edgesPath path to a CSV file with the Edge data
+	* @param context the Flink execution environment.
+	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader}, 
+	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
+	* 
+	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
+	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
+	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
+	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	*/
+	public static GraphCsvReader fromCsvReader(String verticesPath, String edgesPath, ExecutionEnvironment context) {
+		return new GraphCsvReader(verticesPath, edgesPath, context);
+	}
+
+	/** 
+	* Creates a graph from a CSV file of edges. Vertices will be created automatically.
+	*
+	* @param edgesPath a path to a CSV file with the Edges data
+	* @param context the execution environment.
+	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
+	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
+	* 
+	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
+	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
+	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
+	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	*/
+	public static GraphCsvReader fromCsvReader(String edgesPath, ExecutionEnvironment context) {
+		return new GraphCsvReader(edgesPath, context);
+	}
+
+	/** 
+	 * Creates a graph from a CSV file of edges. Vertices will be created automatically and
+	 * Vertex values are set by the provided mapper.
+	 *
+	 * @param edgesPath a path to a CSV file with the Edge data
+	 * @param mapper the mapper function.
+	 * @param context the execution environment.
+	 * @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
+	 * on which calling methods to specify types of the Vertex ID, Vertex Value and Edge value returns a Graph.
+	 * 
+	 * @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
+	 * {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
+	 * {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
+	 * {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	 */
+	public static <K, VV> GraphCsvReader fromCsvReader(String edgesPath,
+			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
+		return new GraphCsvReader(edgesPath, mapper, context);
+	}
+
+	/**
+	 * @return the flink execution environment.
+	 */
+	public ExecutionEnvironment getContext() {
+		return this.context;
+	}
+
+	/**
+	 * Function that checks whether a Graph is a valid Graph,
+	 * as defined by the given {@link GraphValidator}.
+	 * 
+	 * @return true if the Graph is valid.
+	 */
+	public Boolean validate(GraphValidator<K, VV, EV> validator) throws Exception {
+		return validator.validate(this);
+	}
+
+	/**
+	 * @return the vertex DataSet.
+	 */
+	public DataSet<Vertex<K, VV>> getVertices() {
+		return vertices;
+	}
+
+	/**
+	 * @return the edge DataSet.
+	 */
+	public DataSet<Edge<K, EV>> getEdges() {
+		return edges;
+	}
+
+	/**
+	 * @return the vertex DataSet as Tuple2.
+	 */
+	public DataSet<Tuple2<K, VV>> getVerticesAsTuple2() {
+		return vertices.map(new VertexToTuple2Map<K, VV>());
+	}
+
+	/**
+	 * @return the edge DataSet as Tuple3.
+	 */
+	public DataSet<Tuple3<K, K, EV>> getEdgesAsTuple3() {
+		return edges.map(new EdgeToTuple3Map<K, EV>());
+	}
+
+	/**
+	 * This method allows access to the graph's edge values along with its source and target vertex values.
+	 *
+	 * @return a triplet DataSet consisting of (srcVertexId, trgVertexId, srcVertexValue, trgVertexValue, edgeValue)
+	 */
+	public DataSet<Triplet<K, VV, EV>> getTriplets() {
+		return this.getVertices().join(this.getEdges()).where(0).equalTo(0)
+				.with(new ProjectEdgeWithSrcValue<K, VV, EV>())
+				.join(this.getVertices()).where(1).equalTo(0)
+				.with(new ProjectEdgeWithVertexValues<K, VV, EV>());
+	}
+
+	@ForwardedFieldsFirst("f1->f2")
+	@ForwardedFieldsSecond("f0; f1; f2->f3")
+	private static final class ProjectEdgeWithSrcValue<K, VV, EV> implements
+			FlatJoinFunction<Vertex<K, VV>, Edge<K, EV>, Tuple4<K, K, VV, EV>> {
+
+		@Override
+		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple4<K, K, VV, EV>> collector)
+				throws Exception {
+
+			collector.collect(new Tuple4<K, K, VV, EV>(edge.getSource(), edge.getTarget(), vertex.getValue(),
+					edge.getValue()));
+		}
+	}
+
+	@ForwardedFieldsFirst("f0; f1; f2; f3->f4")
+	@ForwardedFieldsSecond("f1->f3")
+	private static final class ProjectEdgeWithVertexValues<K, VV, EV> implements
+			FlatJoinFunction<Tuple4<K, K, VV, EV>, Vertex<K, VV>, Triplet<K, VV, EV>> {
+
+		@Override
+		public void join(Tuple4<K, K, VV, EV> tripletWithSrcValSet,
+						Vertex<K, VV> vertex, Collector<Triplet<K, VV, EV>> collector) throws Exception {
+
+			collector.collect(new Triplet<K, VV, EV>(tripletWithSrcValSet.f0, tripletWithSrcValSet.f1,
+					tripletWithSrcValSet.f2, vertex.getValue(), tripletWithSrcValSet.f3));
+		}
+	}
+
+	/**
+	 * Apply a function to the attribute of each vertex in the graph.
+	 * 
+	 * @param mapper the map function to apply.
+	 * @return a new graph
+	 */
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper) {
+
+		TypeInformation<K> keyType = ((TupleTypeInfo<?>) vertices.getType()).getTypeAt(0);
+
+		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null);
+
+		TypeInformation<Vertex<K, NV>> returnType = (TypeInformation<Vertex<K, NV>>) new TupleTypeInfo(
+				Vertex.class, keyType, valueType);
+
+		return mapVertices(mapper, returnType);
+	}
+
+	/**
+	 * Apply a function to the attribute of each vertex in the graph.
+	 *
+	 * @param mapper the map function to apply.
+	 * @param returnType the explicit return type.
+	 * @return a new graph
+	 */
+	public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper, TypeInformation<Vertex<K,NV>> returnType) {
+		DataSet<Vertex<K, NV>> mappedVertices = vertices.map(
+				new MapFunction<Vertex<K, VV>, Vertex<K, NV>>() {
+					public Vertex<K, NV> map(Vertex<K, VV> value) throws Exception {
+						return new Vertex<K, NV>(value.f0, mapper.map(value));
+					}
+				})
+				.returns(returnType)
+				.withForwardedFields("f0");
+
+		return new Graph<K, NV, EV>(mappedVertices, this.edges, this.context);
+	}
+
+	/**
+	 * Apply a function to the attribute of each edge in the graph.
+	 * 
+	 * @param mapper the map function to apply.
+	 * @return a new graph
+	 */
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper) {
+
+		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
+
+		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null);
+
+		TypeInformation<Edge<K, NV>> returnType = (TypeInformation<Edge<K, NV>>) new TupleTypeInfo(
+				Edge.class, keyType, keyType, valueType);
+
+		return mapEdges(mapper, returnType);
+	}
+
+	/**
+	 * Apply a function to the attribute of each edge in the graph.
+	 *
+	 * @param mapper the map function to apply.
+	 * @param returnType the explicit return type.
+	 * @return a new graph
+	 */
+	public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper, TypeInformation<Edge<K,NV>> returnType) {
+		DataSet<Edge<K, NV>> mappedEdges = edges.map(
+				new MapFunction<Edge<K, EV>, Edge<K, NV>>() {
+					public Edge<K, NV> map(Edge<K, EV> value) throws Exception {
+						return new Edge<K, NV>(value.f0, value.f1, mapper
+								.map(value));
+					}
+				})
+				.returns(returnType)
+				.withForwardedFields("f0; f1");
+
+		return new Graph<K, VV, NV>(this.vertices, mappedEdges, this.context);
+	}
+
+	/**
+	 * Joins the vertex DataSet of this graph with an input DataSet and applies
+	 * a UDF on the resulted values.
+	 * 
+	 * @param inputDataSet the DataSet to join with.
+	 * @param mapper the UDF map function to apply.
+	 * @return a new graph where the vertex values have been updated.
+	 */
+	public <T> Graph<K, VV, EV> joinWithVertices(DataSet<Tuple2<K, T>> inputDataSet, 
+			final MapFunction<Tuple2<VV, T>, VV> mapper) {
+
+		DataSet<Vertex<K, VV>> resultedVertices = this.getVertices()
+				.coGroup(inputDataSet).where(0).equalTo(0)
+				.with(new ApplyCoGroupToVertexValues<K, VV, T>(mapper));
+		return new Graph<K, VV, EV>(resultedVertices, this.edges, this.context);
+	}
+
+	private static final class ApplyCoGroupToVertexValues<K, VV, T>
+			implements CoGroupFunction<Vertex<K, VV>, Tuple2<K, T>, Vertex<K, VV>> {
+
+		private MapFunction<Tuple2<VV, T>, VV> mapper;
+
+		public ApplyCoGroupToVertexValues(MapFunction<Tuple2<VV, T>, VV> mapper) {
+			this.mapper = mapper;
+		}
+
+		@Override
+		public void coGroup(Iterable<Vertex<K, VV>> vertices,
+				Iterable<Tuple2<K, T>> input, Collector<Vertex<K, VV>> collector) throws Exception {
+
+			final Iterator<Vertex<K, VV>> vertexIterator = vertices.iterator();
+			final Iterator<Tuple2<K, T>> inputIterator = input.iterator();
+
+			if (vertexIterator.hasNext()) {
+				if (inputIterator.hasNext()) {
+					final Tuple2<K, T> inputNext = inputIterator.next();
+
+					collector.collect(new Vertex<K, VV>(inputNext.f0, mapper
+							.map(new Tuple2<VV, T>(vertexIterator.next().f1,
+									inputNext.f1))));
+				} else {
+					collector.collect(vertexIterator.next());
+				}
+
+			}
+		}
+	}
+
+	/**
+	 * Joins the edge DataSet with an input DataSet on a composite key of both
+	 * source and target and applies a UDF on the resulted values.
+	 * 
+	 * @param inputDataSet the DataSet to join with.
+	 * @param mapper the UDF map function to apply.
+	 * @param <T> the return type
+	 * @return a new graph where the edge values have been updated.
+	 */
+	public <T> Graph<K, VV, EV> joinWithEdges(DataSet<Tuple3<K, K, T>> inputDataSet,
+			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+
+		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
+				.coGroup(inputDataSet).where(0, 1).equalTo(0, 1)
+				.with(new ApplyCoGroupToEdgeValues<K, EV, T>(mapper));
+		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
+	}
+
+	private static final class ApplyCoGroupToEdgeValues<K, EV, T>
+			implements CoGroupFunction<Edge<K, EV>, Tuple3<K, K, T>, Edge<K, EV>> {
+
+		private MapFunction<Tuple2<EV, T>, EV> mapper;
+
+		public ApplyCoGroupToEdgeValues(MapFunction<Tuple2<EV, T>, EV> mapper) {
+			this.mapper = mapper;
+		}
+
+		@Override
+		public void coGroup(Iterable<Edge<K, EV>> edges, Iterable<Tuple3<K, K, T>> input,
+				Collector<Edge<K, EV>> collector) throws Exception {
+
+			final Iterator<Edge<K, EV>> edgesIterator = edges.iterator();
+			final Iterator<Tuple3<K, K, T>> inputIterator = input.iterator();
+
+			if (edgesIterator.hasNext()) {
+				if (inputIterator.hasNext()) {
+					final Tuple3<K, K, T> inputNext = inputIterator.next();
+
+					collector.collect(new Edge<K, EV>(inputNext.f0,
+							inputNext.f1, mapper.map(new Tuple2<EV, T>(
+									edgesIterator.next().f2, inputNext.f2))));
+				} else {
+					collector.collect(edgesIterator.next());
+				}
+			}
+		}
+	}
+
+	/**
+	 * Joins the edge DataSet with an input DataSet on the source key of the
+	 * edges and the first attribute of the input DataSet and applies a UDF on
+	 * the resulted values. In case the inputDataSet contains the same key more
+	 * than once, only the first value will be considered.
+	 * 
+	 * @param inputDataSet the DataSet to join with.
+	 * @param mapper the UDF map function to apply.
+	 * @param <T> the return type
+	 * @return a new graph where the edge values have been updated.
+	 */
+	public <T> Graph<K, VV, EV> joinWithEdgesOnSource(DataSet<Tuple2<K, T>> inputDataSet,
+			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+
+		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
+				.coGroup(inputDataSet).where(0).equalTo(0)
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
+
+		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
+	}
+
+	private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>
+			implements CoGroupFunction<Edge<K, EV>, Tuple2<K, T>, Edge<K, EV>> {
+
+		private MapFunction<Tuple2<EV, T>, EV> mapper;
+
+		public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(
+				MapFunction<Tuple2<EV, T>, EV> mapper) {
+			this.mapper = mapper;
+		}
+
+		@Override
+		public void coGroup(Iterable<Edge<K, EV>> edges,
+				Iterable<Tuple2<K, T>> input, Collector<Edge<K, EV>> collector) throws Exception {
+
+			final Iterator<Edge<K, EV>> edgesIterator = edges.iterator();
+			final Iterator<Tuple2<K, T>> inputIterator = input.iterator();
+
+			if (inputIterator.hasNext()) {
+				final Tuple2<K, T> inputNext = inputIterator.next();
+
+				while (edgesIterator.hasNext()) {
+					Edge<K, EV> edgesNext = edgesIterator.next();
+
+					collector.collect(new Edge<K, EV>(edgesNext.f0,
+							edgesNext.f1, mapper.map(new Tuple2<EV, T>(
+									edgesNext.f2, inputNext.f1))));
+				}
+
+			} else {
+				while (edgesIterator.hasNext()) {
+					collector.collect(edgesIterator.next());
+				}
+			}
+		}
+	}
+
+	/**
+	 * Joins the edge DataSet with an input DataSet on the target key of the
+	 * edges and the first attribute of the input DataSet and applies a UDF on
+	 * the resulted values. Should the inputDataSet contain the same key more
+	 * than once, only the first value will be considered.
+	 * 
+	 * @param inputDataSet the DataSet to join with.
+	 * @param mapper the UDF map function to apply.
+	 * @param <T> the return type
+	 * @return a new graph where the edge values have been updated.
+	 */
+	public <T> Graph<K, VV, EV> joinWithEdgesOnTarget(DataSet<Tuple2<K, T>> inputDataSet,
+			final MapFunction<Tuple2<EV, T>, EV> mapper) {
+
+		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
+				.coGroup(inputDataSet).where(1).equalTo(0)
+				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
+
+		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
+	}
+
+	/**
+	 * Apply filtering functions to the graph and return a sub-graph that
+	 * satisfies the predicates for both vertices and edges.
+	 * 
+	 * @param vertexFilter the filter function for vertices.
+	 * @param edgeFilter the filter function for edges.
+	 * @return the resulting sub-graph.
+	 */
+	public Graph<K, VV, EV> subgraph(FilterFunction<Vertex<K, VV>> vertexFilter, FilterFunction<Edge<K, EV>> edgeFilter) {
+
+		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
+
+		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
+				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
+				.join(filteredVertices).where(1).equalTo(0)
+				.with(new ProjectEdge<K, VV, EV>());
+
+		DataSet<Edge<K, EV>> filteredEdges = remainingEdges.filter(edgeFilter);
+
+		return new Graph<K, VV, EV>(filteredVertices, filteredEdges,
+				this.context);
+	}
+
+	/**
+	 * Apply a filtering function to the graph and return a sub-graph that
+	 * satisfies the predicates only for the vertices.
+	 * 
+	 * @param vertexFilter the filter function for vertices.
+	 * @return the resulting sub-graph.
+	 */
+	public Graph<K, VV, EV> filterOnVertices(FilterFunction<Vertex<K, VV>> vertexFilter) {
+
+		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
+
+		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
+				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
+				.join(filteredVertices).where(1).equalTo(0)
+				.with(new ProjectEdge<K, VV, EV>());
+
+		return new Graph<K, VV, EV>(filteredVertices, remainingEdges, this.context);
+	}
+
+	/**
+	 * Apply a filtering function to the graph and return a sub-graph that
+	 * satisfies the predicates only for the edges.
+	 * 
+	 * @param edgeFilter the filter function for edges.
+	 * @return the resulting sub-graph.
+	 */
+	public Graph<K, VV, EV> filterOnEdges(FilterFunction<Edge<K, EV>> edgeFilter) {
+		DataSet<Edge<K, EV>> filteredEdges = this.edges.filter(edgeFilter);
+
+		return new Graph<K, VV, EV>(this.vertices, filteredEdges, this.context);
+	}
+
+	@ForwardedFieldsFirst("f0; f1; f2")
+	private static final class ProjectEdge<K, VV, EV> implements FlatJoinFunction<
+		Edge<K, EV>, Vertex<K, VV>, Edge<K, EV>> {
+		public void join(Edge<K, EV> first, Vertex<K, VV> second, Collector<Edge<K, EV>> out) {
+			out.collect(first);
+		}
+	}
+
+	/**
+	 * Return the out-degree of all vertices in the graph
+	 * 
+	 * @return A DataSet of Tuple2<vertexId, outDegree>
+	 */
+	public DataSet<Tuple2<K, Long>> outDegrees() {
+
+		return vertices.coGroup(edges).where(0).equalTo(0).with(new CountNeighborsCoGroup<K, VV, EV>());
+	}
+
+	private static final class CountNeighborsCoGroup<K, VV, EV>
+			implements CoGroupFunction<Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Long>> {
+		@SuppressWarnings("unused")
+		public void coGroup(Iterable<Vertex<K, VV>> vertex,	Iterable<Edge<K, EV>> outEdges,
+				Collector<Tuple2<K, Long>> out) {
+			long count = 0;
+			for (Edge<K, EV> edge : outEdges) {
+				count++;
+			}
+
+			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
+
+			if(vertexIterator.hasNext()) {
+				out.collect(new Tuple2<K, Long>(vertexIterator.next().f0, count));
+			} else {
+				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
+			}
+		}
+	}
+
+	/**
+	 * Return the in-degree of all vertices in the graph
+	 * 
+	 * @return A DataSet of Tuple2<vertexId, inDegree>
+	 */
+	public DataSet<Tuple2<K, Long>> inDegrees() {
+
+		return vertices.coGroup(edges).where(0).equalTo(1).with(new CountNeighborsCoGroup<K, VV, EV>());
+	}
+
+	/**
+	 * Return the degree of all vertices in the graph
+	 * 
+	 * @return A DataSet of Tuple2<vertexId, degree>
+	 */
+	public DataSet<Tuple2<K, Long>> getDegrees() {
+		return outDegrees().union(inDegrees()).groupBy(0).sum(1);
+	}
+
+	/**
+	 * This operation adds all inverse-direction edges to the graph.
+	 * 
+	 * @return the undirected graph.
+	 */
+	public Graph<K, VV, EV> getUndirected() {
+
+		DataSet<Edge<K, EV>> undirectedEdges = edges.flatMap(new RegularAndReversedEdgesMap<K, EV>());
+		return new Graph<K, VV, EV>(vertices, undirectedEdges, this.context);
+	}
+
+	/**
+	 * Compute an aggregate over the edges of each vertex. The function applied
+	 * on the edges has access to the vertex value.
+	 * 
+	 * @param edgesFunction
+	 *            the function to apply to the neighborhood
+	 * @param direction
+	 *            the edge direction (in-, out-, all-)
+	 * @param <T>
+	 *            the output type
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
+											EdgeDirection direction) throws IllegalArgumentException {
+
+		switch (direction) {
+		case IN:
+			return vertices.coGroup(edges).where(0).equalTo(1)
+					.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction));
+		case OUT:
+			return vertices.coGroup(edges).where(0).equalTo(0)
+					.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction));
+		case ALL:
+			return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>()))
+					.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>(edgesFunction));
+		default:
+			throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the edges of each vertex. The function applied
+	 * on the edges has access to the vertex value.
+	 *
+	 * @param edgesFunction
+	 *            the function to apply to the neighborhood
+	 * @param direction
+	 *            the edge direction (in-, out-, all-)
+	 * @param <T>
+	 *            the output type
+	 * @param typeInfo the explicit return type.
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
+											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+
+		switch (direction) {
+			case IN:
+				return vertices.coGroup(edges).where(0).equalTo(1)
+						.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
+			case OUT:
+				return vertices.coGroup(edges).where(0).equalTo(0)
+						.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
+			case ALL:
+				return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>()))
+						.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the edges of each vertex. The function applied
+	 * on the edges only has access to the vertex id (not the vertex value).
+	 * 
+	 * @param edgesFunction
+	 *            the function to apply to the neighborhood
+	 * @param direction
+	 *            the edge direction (in-, out-, all-)
+	 * @param <T>
+	 *            the output type
+	 * @return a dataset of T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
+											EdgeDirection direction) throws IllegalArgumentException {
+
+		switch (direction) {
+		case IN:
+			return edges.map(new ProjectVertexIdMap<K, EV>(1))
+					.withForwardedFields("f1->f0")
+					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
+		case OUT:
+			return edges.map(new ProjectVertexIdMap<K, EV>(0))
+					.withForwardedFields("f0")
+					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
+		case ALL:
+			return edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>())
+					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
+		default:
+			throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the edges of each vertex. The function applied
+	 * on the edges only has access to the vertex id (not the vertex value).
+	 *
+	 * @param edgesFunction
+	 *            the function to apply to the neighborhood
+	 * @param direction
+	 *            the edge direction (in-, out-, all-)
+	 * @param <T>
+	 *            the output type
+	 * @param typeInfo the explicit return type.
+	 * @return a dataset of T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
+											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+
+		switch (direction) {
+			case IN:
+				return edges.map(new ProjectVertexIdMap<K, EV>(1))
+						.withForwardedFields("f1->f0")
+						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
+			case OUT:
+				return edges.map(new ProjectVertexIdMap<K, EV>(0))
+						.withForwardedFields("f0")
+						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
+			case ALL:
+				return edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>())
+						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	private static final class ProjectVertexIdMap<K, EV> implements MapFunction<
+		Edge<K, EV>, Tuple2<K, Edge<K, EV>>> {
+
+		private int fieldPosition;
+
+		public ProjectVertexIdMap(int position) {
+			this.fieldPosition = position;
+		}
+
+		@SuppressWarnings("unchecked")
+		public Tuple2<K, Edge<K, EV>> map(Edge<K, EV> edge) {
+			return new Tuple2<K, Edge<K, EV>>((K) edge.getField(fieldPosition),	edge);
+		}
+	}
+
+	private static final class ProjectVertexWithEdgeValueMap<K, EV>	implements MapFunction<
+		Edge<K, EV>, Tuple2<K, EV>> {
+
+		private int fieldPosition;
+
+		public ProjectVertexWithEdgeValueMap(int position) {
+			this.fieldPosition = position;
+		}
+
+		@SuppressWarnings("unchecked")
+		public Tuple2<K, EV> map(Edge<K, EV> edge) {
+			return new Tuple2<K, EV>((K) edge.getField(fieldPosition),	edge.getValue());
+		}
+	}
+
+	private static final class ApplyGroupReduceFunction<K, EV, T> implements GroupReduceFunction<
+		Tuple2<K, Edge<K, EV>>, T>,	ResultTypeQueryable<T> {
+
+		private EdgesFunction<K, EV, T> function;
+
+		public ApplyGroupReduceFunction(EdgesFunction<K, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void reduce(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<T> out) throws Exception {
+			function.iterateEdges(edges, out);
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null);
+		}
+	}
+
+	private static final class EmitOneEdgePerNode<K, VV, EV> implements FlatMapFunction<
+		Edge<K, EV>, Tuple2<K, Edge<K, EV>>> {
+
+		public void flatMap(Edge<K, EV> edge, Collector<Tuple2<K, Edge<K, EV>>> out) {
+			out.collect(new Tuple2<K, Edge<K, EV>>(edge.getSource(), edge));
+			out.collect(new Tuple2<K, Edge<K, EV>>(edge.getTarget(), edge));
+		}
+	}
+
+	private static final class EmitOneVertexWithEdgeValuePerNode<K, EV>	implements FlatMapFunction<
+		Edge<K, EV>, Tuple2<K, EV>> {
+
+		public void flatMap(Edge<K, EV> edge, Collector<Tuple2<K, EV>> out) {
+			out.collect(new Tuple2<K, EV>(edge.getSource(), edge.getValue()));
+			out.collect(new Tuple2<K, EV>(edge.getTarget(), edge.getValue()));
+		}
+	}
+
+	private static final class EmitOneEdgeWithNeighborPerNode<K, EV> implements FlatMapFunction<
+		Edge<K, EV>, Tuple3<K, K, Edge<K, EV>>> {
+
+		public void flatMap(Edge<K, EV> edge, Collector<Tuple3<K, K, Edge<K, EV>>> out) {
+			out.collect(new Tuple3<K, K, Edge<K, EV>>(edge.getSource(), edge.getTarget(), edge));
+			out.collect(new Tuple3<K, K, Edge<K, EV>>(edge.getTarget(), edge.getSource(), edge));
+		}
+	}
+
+	private static final class ApplyCoGroupFunction<K, VV, EV, T> implements CoGroupFunction<
+		Vertex<K, VV>, Edge<K, EV>, T>, ResultTypeQueryable<T> {
+
+		private EdgesFunctionWithVertexValue<K, VV, EV, T> function;
+
+		public ApplyCoGroupFunction(EdgesFunctionWithVertexValue<K, VV, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void coGroup(Iterable<Vertex<K, VV>> vertex,
+				Iterable<Edge<K, EV>> edges, Collector<T> out) throws Exception {
+
+			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
+
+			if(vertexIterator.hasNext()) {
+				function.iterateEdges(vertexIterator.next(), edges, out);
+			} else {
+				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
+			}
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3,
+					null, null);
+		}
+	}
+
+	private static final class ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>
+			implements	CoGroupFunction<Vertex<K, VV>, Tuple2<K, Edge<K, EV>>, T>, ResultTypeQueryable<T> {
+
+		private EdgesFunctionWithVertexValue<K, VV, EV, T> function;
+
+		public ApplyCoGroupFunctionOnAllEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void coGroup(Iterable<Vertex<K, VV>> vertex,	final Iterable<Tuple2<K, Edge<K, EV>>> keysWithEdges,
+				Collector<T> out) throws Exception {
+
+			final Iterator<Edge<K, EV>> edgesIterator = new Iterator<Edge<K, EV>>() {
+
+				final Iterator<Tuple2<K, Edge<K, EV>>> keysWithEdgesIterator = keysWithEdges.iterator();
+
+				@Override
+				public boolean hasNext() {
+					return keysWithEdgesIterator.hasNext();
+				}
+
+				@Override
+				public Edge<K, EV> next() {
+					return keysWithEdgesIterator.next().f1;
+				}
+
+				@Override
+				public void remove() {
+					keysWithEdgesIterator.remove();
+				}
+			};
+
+			Iterable<Edge<K, EV>> edgesIterable = new Iterable<Edge<K, EV>>() {
+				public Iterator<Edge<K, EV>> iterator() {
+					return edgesIterator;
+				}
+			};
+
+			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
+
+			if(vertexIterator.hasNext()) {
+				function.iterateEdges(vertexIterator.next(), edgesIterable, out);
+			} else {
+				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
+			}
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3,
+					null, null);
+		}
+	}
+
+	@ForwardedFields("f0->f1; f1->f0; f2")
+	private static final class ReverseEdgesMap<K, EV>
+			implements MapFunction<Edge<K, EV>, Edge<K, EV>> {
+
+		public Edge<K, EV> map(Edge<K, EV> value) {
+			return new Edge<K, EV>(value.f1, value.f0, value.f2);
+		}
+	}
+
+	private static final class RegularAndReversedEdgesMap<K, EV>
+			implements FlatMapFunction<Edge<K, EV>, Edge<K, EV>> {
+
+		@Override
+		public void flatMap(Edge<K, EV> edge, Collector<Edge<K, EV>> out) throws Exception {
+			out.collect(new Edge<K, EV>(edge.f0, edge.f1, edge.f2));
+			out.collect(new Edge<K, EV>(edge.f1, edge.f0, edge.f2));
+		}
+	}
+
+	/**
+	 * Reverse the direction of the edges in the graph
+	 * 
+	 * @return a new graph with all edges reversed
+	 * @throws UnsupportedOperationException
+	 */
+	public Graph<K, VV, EV> reverse() throws UnsupportedOperationException {
+		DataSet<Edge<K, EV>> reversedEdges = edges.map(new ReverseEdgesMap<K, EV>());
+		return new Graph<K, VV, EV>(vertices, reversedEdges, this.context);
+	}
+
+	/**
+	 * @return a long integer representing the number of vertices
+	 */
+	public long numberOfVertices() throws Exception {
+		return vertices.count();
+	}
+
+	/**
+	 * @return a long integer representing the number of edges
+	 */
+	public long numberOfEdges() throws Exception {
+		return edges.count();
+	}
+
+	/**
+	 * @return The IDs of the vertices as DataSet
+	 */
+	public DataSet<K> getVertexIds() {
+		return vertices.map(new ExtractVertexIDMapper<K, VV>());
+	}
+
+	private static final class ExtractVertexIDMapper<K, VV>
+			implements MapFunction<Vertex<K, VV>, K> {
+		@Override
+		public K map(Vertex<K, VV> vertex) {
+			return vertex.f0;
+		}
+	}
+
+	/**
+	 * @return The IDs of the edges as DataSet
+	 */
+	public DataSet<Tuple2<K, K>> getEdgeIds() {
+		return edges.map(new ExtractEdgeIDsMapper<K, EV>());
+	}
+
+	@ForwardedFields("f0; f1")
+	private static final class ExtractEdgeIDsMapper<K, EV>
+			implements MapFunction<Edge<K, EV>, Tuple2<K, K>> {
+		@Override
+		public Tuple2<K, K> map(Edge<K, EV> edge) throws Exception {
+			return new Tuple2<K, K>(edge.f0, edge.f1);
+		}
+	}
+
+	/**
+	 * Adds the input vertex to the graph. If the vertex already
+	 * exists in the graph, it will not be added again.
+	 * 
+	 * @param vertex the vertex to be added
+	 * @return the new graph containing the existing vertices as well as the one just added
+	 */
+	public Graph<K, VV, EV> addVertex(final Vertex<K, VV> vertex) {
+		List<Vertex<K, VV>> newVertex = new ArrayList<Vertex<K, VV>>();
+		newVertex.add(vertex);
+
+		return addVertices(newVertex);
+	}
+
+	/**
+	 * Adds the list of vertices, passed as input, to the graph.
+	 * If the vertices already exist in the graph, they will not be added once more.
+	 *
+	 * @param verticesToAdd the list of vertices to add
+	 * @return the new graph containing the existing and newly added vertices
+	 */
+	public Graph<K, VV, EV> addVertices(List<Vertex<K, VV>> verticesToAdd) {
+		// Add the vertices
+		DataSet<Vertex<K, VV>> newVertices = this.vertices.union(this.context.fromCollection(verticesToAdd)).distinct();
+
+		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
+	}
+
+	/**
+	 * Adds the given edge to the graph. If the source and target vertices do
+	 * not exist in the graph, they will also be added.
+	 * 
+	 * @param source the source vertex of the edge
+	 * @param target the target vertex of the edge
+	 * @param edgeValue the edge value
+	 * @return the new graph containing the existing vertices and edges plus the
+	 *         newly added edge
+	 */
+	public Graph<K, VV, EV> addEdge(Vertex<K, VV> source, Vertex<K, VV> target, EV edgeValue) {
+		Graph<K, VV, EV> partialGraph = fromCollection(Arrays.asList(source, target),
+				Arrays.asList(new Edge<K, EV>(source.f0, target.f0, edgeValue)),
+				this.context);
+		return this.union(partialGraph);
+	}
+
+	/**
+	 * Adds the given list edges to the graph.
+	 *
+	 * When adding an edge for a non-existing set of vertices, the edge is considered invalid and ignored.
+	 *
+	 * @param newEdges the data set of edges to be added
+	 * @return a new graph containing the existing edges plus the newly added edges.
+	 */
+	public Graph<K, VV, EV> addEdges(List<Edge<K, EV>> newEdges) {
+
+		DataSet<Edge<K,EV>> newEdgesDataSet = this.context.fromCollection(newEdges);
+
+		DataSet<Edge<K,EV>> validNewEdges = this.getVertices().join(newEdgesDataSet)
+				.where(0).equalTo(0)
+				.with(new JoinVerticesWithEdgesOnSrc<K, VV, EV>())
+				.join(this.getVertices()).where(1).equalTo(0)
+				.with(new JoinWithVerticesOnTrg<K, VV, EV>());
+
+		return Graph.fromDataSet(this.vertices, this.edges.union(validNewEdges), this.context);
+	}
+
+	@ForwardedFieldsSecond("f0; f1; f2")
+	private static final class JoinVerticesWithEdgesOnSrc<K, VV, EV> implements
+			JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
+
+		@Override
+		public Edge<K, EV> join(Vertex<K, VV> vertex, Edge<K, EV> edge) throws Exception {
+			return edge;
+		}
+	}
+
+	@ForwardedFieldsFirst("f0; f1; f2")
+	private static final class JoinWithVerticesOnTrg<K, VV, EV> implements
+			JoinFunction<Edge<K, EV>, Vertex<K, VV>, Edge<K, EV>> {
+
+		@Override
+		public Edge<K, EV> join(Edge<K, EV> edge, Vertex<K, VV> vertex) throws Exception {
+			return edge;
+		}
+	}
+
+	/**
+	 * Removes the given vertex and its edges from the graph.
+	 * 
+	 * @param vertex the vertex to remove
+	 * @return the new graph containing the existing vertices and edges without
+	 *         the removed vertex and its edges
+	 */
+	public Graph<K, VV, EV> removeVertex(Vertex<K, VV> vertex) {
+
+		List<Vertex<K, VV>> vertexToBeRemoved = new ArrayList<Vertex<K, VV>>();
+		vertexToBeRemoved.add(vertex);
+
+		return removeVertices(vertexToBeRemoved);
+	}
+	/**
+	 * Removes the given list of vertices and its edges from the graph.
+	 *
+	 * @param verticesToBeRemoved the list of vertices to be removed
+	 * @return the resulted graph containing the initial vertices and edges minus the vertices
+	 * 		   and edges removed.
+	 */
+
+	public Graph<K, VV, EV> removeVertices(List<Vertex<K, VV>> verticesToBeRemoved)
+	{
+		return removeVertices(this.context.fromCollection(verticesToBeRemoved));
+	}
+
+	/**
+	 * Removes the given list of vertices and its edges from the graph.
+	 *
+	 * @param verticesToBeRemoved the DataSet of vertices to be removed
+	 * @return the resulted graph containing the initial vertices and edges minus the vertices
+	 * 		   and edges removed.
+	 */
+	private Graph<K, VV, EV> removeVertices(DataSet<Vertex<K, VV>> verticesToBeRemoved) {
+
+		DataSet<Vertex<K, VV>> newVertices = getVertices().coGroup(verticesToBeRemoved).where(0).equalTo(0)
+				.with(new VerticesRemovalCoGroup<K, VV>());
+
+		DataSet < Edge < K, EV >> newEdges = newVertices.join(getEdges()).where(0).equalTo(0)
+				// if the edge source was removed, the edge will also be removed
+				.with(new ProjectEdgeToBeRemoved<K, VV, EV>())
+				// if the edge target was removed, the edge will also be removed
+				.join(newVertices).where(1).equalTo(0)
+				.with(new ProjectEdge<K, VV, EV>());
+
+		return new Graph<K, VV, EV>(newVertices, newEdges, context);
+	}
+
+	private static final class VerticesRemovalCoGroup<K, VV> implements CoGroupFunction<Vertex<K, VV>, Vertex<K, VV>, Vertex<K, VV>> {
+
+		@Override
+		public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Vertex<K, VV>> vertexToBeRemoved,
+							Collector<Vertex<K, VV>> out) throws Exception {
+
+			final Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
+			final Iterator<Vertex<K, VV>> vertexToBeRemovedIterator = vertexToBeRemoved.iterator();
+			Vertex<K, VV> next;
+
+			if (vertexIterator.hasNext()) {
+				if (!vertexToBeRemovedIterator.hasNext()) {
+					next = vertexIterator.next();
+					out.collect(next);
+				}
+			}
+		}
+	}
+
+
+
+	@ForwardedFieldsSecond("f0; f1; f2")
+	private static final class ProjectEdgeToBeRemoved<K,VV,EV> implements JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
+		@Override
+		public Edge<K, EV> join(Vertex<K, VV> vertex, Edge<K, EV> edge) throws Exception {
+			return edge;
+		}
+	}
+
+	 /**
+	 * Removes all edges that match the given edge from the graph.
+	 * 
+	 * @param edge the edge to remove
+	 * @return the new graph containing the existing vertices and edges without
+	 *         the removed edges
+	 */
+	public Graph<K, VV, EV> removeEdge(Edge<K, EV> edge) {
+		DataSet<Edge<K, EV>> newEdges = getEdges().filter(new EdgeRemovalEdgeFilter<K, EV>(edge));
+		return new Graph<K, VV, EV>(this.vertices, newEdges, this.context);
+	}
+
+	private static final class EdgeRemovalEdgeFilter<K, EV>
+			implements FilterFunction<Edge<K, EV>> {
+		private Edge<K, EV> edgeToRemove;
+
+		public EdgeRemovalEdgeFilter(Edge<K, EV> edge) {
+			edgeToRemove = edge;
+		}
+
+		@Override
+		public boolean filter(Edge<K, EV> edge) {
+			return (!(edge.f0.equals(edgeToRemove.f0) && edge.f1
+					.equals(edgeToRemove.f1)));
+		}
+	}
+
+	/**
+	 * Removes all the edges that match the edges in the given data set from the graph.
+	 *
+	 * @param edgesToBeRemoved the list of edges to be removed
+	 * @return a new graph where the edges have been removed and in which the vertices remained intact
+	 */
+	public Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved) {
+
+		DataSet<Edge<K, EV>> newEdges = getEdges().coGroup(this.context.fromCollection(edgesToBeRemoved))
+				.where(0,1).equalTo(0,1).with(new EdgeRemovalCoGroup<K, EV>());
+
+		return new Graph<K, VV, EV>(this.vertices, newEdges, context);
+	}
+
+	private static final class EdgeRemovalCoGroup<K,EV> implements CoGroupFunction<Edge<K, EV>, Edge<K, EV>, Edge<K, EV>> {
+
+		@Override
+		public void coGroup(Iterable<Edge<K, EV>> edge, Iterable<Edge<K, EV>> edgeToBeRemoved,
+							Collector<Edge<K, EV>> out) throws Exception {
+
+			final Iterator<Edge<K, EV>> edgeIterator = edge.iterator();
+			final Iterator<Edge<K, EV>> edgeToBeRemovedIterator = edgeToBeRemoved.iterator();
+			Edge<K, EV> next;
+
+			if (edgeIterator.hasNext()) {
+				if (!edgeToBeRemovedIterator.hasNext()) {
+					next = edgeIterator.next();
+					out.collect(next);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Performs union on the vertices and edges sets of the input graphs
+	 * removing duplicate vertices but maintaining duplicate edges.
+	 * 
+	 * @param graph the graph to perform union with
+	 * @return a new graph
+	 */
+	public Graph<K, VV, EV> union(Graph<K, VV, EV> graph) {
+
+		DataSet<Vertex<K, VV>> unionedVertices = graph.getVertices().union(this.getVertices()).distinct();
+		DataSet<Edge<K, EV>> unionedEdges = graph.getEdges().union(this.getEdges());
+		return new Graph<K, VV, EV>(unionedVertices, unionedEdges, this.context);
+	}
+
+	/**
+	 * Performs Difference on the vertex and edge sets of the input graphs
+	 * removes common vertices and edges. If a source/target vertex is removed, its corresponding edge will also be removed
+	 * @param graph the graph to perform difference with
+	 * @return a new graph where the common vertices and edges have been removed
+	 */
+	public Graph<K,VV,EV> difference(Graph<K,VV,EV> graph) {
+		DataSet<Vertex<K,VV>> removeVerticesData = graph.getVertices();
+		return this.removeVertices(removeVerticesData);
+	}
+
+	/**
+	 * Runs a Vertex-Centric iteration on the graph.
+	 * No configuration options are provided.
+	 *
+	 * @param vertexUpdateFunction the vertex update function
+	 * @param messagingFunction the messaging function
+	 * @param maximumNumberOfIterations maximum number of iterations to perform
+	 * 
+	 * @return the updated Graph after the vertex-centric iteration has converged or
+	 * after maximumNumberOfIterations.
+	 */
+	public <M> Graph<K, VV, EV> runVertexCentricIteration(
+			VertexUpdateFunction<K, VV, M> vertexUpdateFunction,
+			MessagingFunction<K, VV, M, EV> messagingFunction,
+			int maximumNumberOfIterations) {
+
+		return this.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
+				maximumNumberOfIterations, null);
+	}
+
+	/**
+	 * Runs a Vertex-Centric iteration on the graph with configuration options.
+	 * 
+	 * @param vertexUpdateFunction the vertex update function
+	 * @param messagingFunction the messaging function
+	 * @param maximumNumberOfIterations maximum number of iterations to perform
+	 * @param parameters the iteration configuration parameters
+	 * 
+	 * @return the updated Graph after the vertex-centric iteration has converged or
+	 * after maximumNumberOfIterations.
+	 */
+	public <M> Graph<K, VV, EV> runVertexCentricIteration(
+			VertexUpdateFunction<K, VV, M> vertexUpdateFunction,
+			MessagingFunction<K, VV, M, EV> messagingFunction,
+			int maximumNumberOfIterations, VertexCentricConfiguration parameters) {
+
+		VertexCentricIteration<K, VV, M, EV> iteration = VertexCentricIteration.withEdges(
+				edges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations);
+
+		iteration.configure(parameters);
+
+		DataSet<Vertex<K, VV>> newVertices = this.getVertices().runOperation(iteration);
+
+		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
+	}
+
+	/**
+	 * Runs a Gather-Sum-Apply iteration on the graph.
+	 * No configuration options are provided.
+	 *
+	 * @param gatherFunction the gather function collects information about adjacent vertices and edges
+	 * @param sumFunction the sum function aggregates the gathered information
+	 * @param applyFunction the apply function updates the vertex values with the aggregates
+	 * @param maximumNumberOfIterations maximum number of iterations to perform
+	 * @param <M> the intermediate type used between gather, sum and apply
+	 *
+	 * @return the updated Graph after the gather-sum-apply iteration has converged or
+	 * after maximumNumberOfIterations.
+	 */
+	public <M> Graph<K, VV, EV> runGatherSumApplyIteration(
+			GatherFunction<VV, EV, M> gatherFunction, SumFunction<VV, EV, M> sumFunction,
+			ApplyFunction<K, VV, M> applyFunction, int maximumNumberOfIterations) {
+
+		return this.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
+				maximumNumberOfIterations, null);
+	}
+
+	/**
+	 * Runs a Gather-Sum-Apply iteration on the graph with configuration options.
+	 *
+	 * @param gatherFunction the gather function collects information about adjacent vertices and edges
+	 * @param sumFunction the sum function aggregates the gathered information
+	 * @param applyFunction the apply function updates the vertex values with the aggregates
+	 * @param maximumNumberOfIterations maximum number of iterations to perform
+	 * @param parameters the iteration configuration parameters
+	 * @param <M> the intermediate type used between gather, sum and apply
+	 *
+	 * @return the updated Graph after the gather-sum-apply iteration has converged or
+	 * after maximumNumberOfIterations.
+	 */
+	public <M> Graph<K, VV, EV> runGatherSumApplyIteration(
+			GatherFunction<VV, EV, M> gatherFunction, SumFunction<VV, EV, M> sumFunction,
+			ApplyFunction<K, VV, M> applyFunction, int maximumNumberOfIterations,
+			GSAConfiguration parameters) {
+
+		GatherSumApplyIteration<K, VV, EV, M> iteration = GatherSumApplyIteration.withEdges(
+				edges, gatherFunction, sumFunction, applyFunction, maximumNumberOfIterations);
+
+		iteration.configure(parameters);
+
+		DataSet<Vertex<K, VV>> newVertices = vertices.runOperation(iteration);
+
+		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
+	}
+
+	/**
+	 * @param algorithm the algorithm to run on the Graph
+	 * @param <T> the return type
+	 * @return the result of the graph algorithm
+	 * @throws Exception
+	 */
+	public <T> T run(GraphAlgorithm<K, VV, EV, T> algorithm) throws Exception {
+		return algorithm.run(this);
+	}
+
+	/**
+	 * Compute an aggregate over the neighbors (edges and vertices) of each
+	 * vertex. The function applied on the neighbors has access to the vertex
+	 * value.
+	 * 
+	 * @param neighborsFunction the function to apply to the neighborhood
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @param <T> the output type
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
+												EdgeDirection direction) throws IllegalArgumentException {
+		switch (direction) {
+		case IN:
+			// create <edge-sourceVertex> pairs
+			DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
+					.join(this.vertices).where(0).equalTo(0);
+			return vertices.coGroup(edgesWithSources)
+					.where(0).equalTo("f0.f1")
+					.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction));
+		case OUT:
+			// create <edge-targetVertex> pairs
+			DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
+					.join(this.vertices).where(1).equalTo(0);
+			return vertices.coGroup(edgesWithTargets)
+					.where(0).equalTo("f0.f0")
+					.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction));
+		case ALL:
+			// create <edge-sourceOrTargetVertex> pairs
+			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
+					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+					.join(this.vertices).where(1).equalTo(0)
+					.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
+
+			return vertices.coGroup(edgesWithNeighbors)
+					.where(0).equalTo(0)
+					.with(new ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>(neighborsFunction));
+		default:
+			throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the neighbors (edges and vertices) of each
+	 * vertex. The function applied on the neighbors has access to the vertex
+	 * value.
+	 *
+	 * @param neighborsFunction the function to apply to the neighborhood
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @param <T> the output type
+	 * @param typeInfo the explicit return type.
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
+												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+		switch (direction) {
+			case IN:
+				// create <edge-sourceVertex> pairs
+				DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
+						.join(this.vertices).where(0).equalTo(0);
+				return vertices.coGroup(edgesWithSources)
+						.where(0).equalTo("f0.f1")
+						.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			case OUT:
+				// create <edge-targetVertex> pairs
+				DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
+						.join(this.vertices).where(1).equalTo(0);
+				return vertices.coGroup(edgesWithTargets)
+						.where(0).equalTo("f0.f0")
+						.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			case ALL:
+				// create <edge-sourceOrTargetVertex> pairs
+				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+						.join(this.vertices).where(1).equalTo(0)
+						.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
+
+				return vertices.coGroup(edgesWithNeighbors)
+						.where(0).equalTo(0)
+						.with(new ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+
+	/**
+	 * Compute an aggregate over the neighbors (edges and vertices) of each
+	 * vertex. The function applied on the neighbors only has access to the
+	 * vertex id (not the vertex value).
+	 * 
+	 * @param neighborsFunction the function to apply to the neighborhood
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @param <T> the output type
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
+												EdgeDirection direction) throws IllegalArgumentException {
+		switch (direction) {
+		case IN:
+			// create <edge-sourceVertex> pairs
+			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
+					.join(this.vertices).where(0).equalTo(0)
+					.with(new ProjectVertexIdJoin<K, VV, EV>(1))
+					.withForwardedFieldsFirst("f1->f0");
+			return edgesWithSources.groupBy(0).reduceGroup(
+					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
+		case OUT:
+			// create <edge-targetVertex> pairs
+			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
+					.join(this.vertices).where(1).equalTo(0)
+					.with(new ProjectVertexIdJoin<K, VV, EV>(0))
+					.withForwardedFieldsFirst("f0");
+			return edgesWithTargets.groupBy(0).reduceGroup(
+					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
+		case ALL:
+			// create <edge-sourceOrTargetVertex> pairs
+			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
+					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+					.join(this.vertices).where(1).equalTo(0)
+					.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
+
+			return edgesWithNeighbors.groupBy(0).reduceGroup(
+					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
+		default:
+			throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the neighbors (edges and vertices) of each
+	 * vertex. The function applied on the neighbors only has access to the
+	 * vertex id (not the vertex value).
+	 *
+	 * @param neighborsFunction the function to apply to the neighborhood
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @param <T> the output type
+	 * @param typeInfo the explicit return type.
+	 * @return a dataset of a T
+	 * @throws IllegalArgumentException
+	 */
+	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
+												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
+		switch (direction) {
+			case IN:
+				// create <edge-sourceVertex> pairs
+				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
+						.join(this.vertices).where(0).equalTo(0)
+						.with(new ProjectVertexIdJoin<K, VV, EV>(1))
+						.withForwardedFieldsFirst("f1->f0");
+				return edgesWithSources.groupBy(0).reduceGroup(
+						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			case OUT:
+				// create <edge-targetVertex> pairs
+				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
+						.join(this.vertices).where(1).equalTo(0)
+						.with(new ProjectVertexIdJoin<K, VV, EV>(0))
+						.withForwardedFieldsFirst("f0");
+				return edgesWithTargets.groupBy(0).reduceGroup(
+						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			case ALL:
+				// create <edge-sourceOrTargetVertex> pairs
+				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+						.join(this.vertices).where(1).equalTo(0)
+						.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
+
+				return edgesWithNeighbors.groupBy(0).reduceGroup(
+						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	private static final class ApplyNeighborGroupReduceFunction<K, VV, EV, T>
+			implements GroupReduceFunction<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
+
+		private NeighborsFunction<K, VV, EV, T> function;
+
+		public ApplyNeighborGroupReduceFunction(NeighborsFunction<K, VV, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void reduce(Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edges, Collector<T> out) throws Exception {
+			function.iterateNeighbors(edges, out);
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(NeighborsFunction.class, function.getClass(), 3, null, null);
+		}
+	}
+
+	@ForwardedFieldsSecond("f1")
+	private static final class ProjectVertexWithNeighborValueJoin<K, VV, EV>
+			implements FlatJoinFunction<Edge<K, EV>, Vertex<K, VV>, Tuple2<K, VV>> {
+
+		private int fieldPosition;
+
+		public ProjectVertexWithNeighborValueJoin(int position) {
+			this.fieldPosition = position;
+		}
+
+		@SuppressWarnings("unchecked")
+		public void join(Edge<K, EV> edge, Vertex<K, VV> otherVertex, 
+				Collector<Tuple2<K, VV>> out) {
+			out.collect(new Tuple2<K, VV>((K) edge.getField(fieldPosition), otherVertex.getValue()));
+		}
+	}
+
+	private static final class ProjectVertexIdJoin<K, VV, EV> implements FlatJoinFunction<
+		Edge<K, EV>, Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> {
+
+		private int fieldPosition;
+
+		public ProjectVertexIdJoin(int position) {
+			this.fieldPosition = position;
+		}
+
+		@SuppressWarnings("unchecked")
+		public void join(Edge<K, EV> edge, Vertex<K, VV> otherVertex,
+						Collector<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> out) {
+			out.collect(new Tuple3<K, Edge<K, EV>, Vertex<K, VV>>((K) edge.getField(fieldPosition), edge, otherVertex));
+		}
+	}
+
+	@ForwardedFieldsFirst("f0")
+	@ForwardedFieldsSecond("f1")
+	private static final class ProjectNeighborValue<K, VV, EV> implements FlatJoinFunction<
+		Tuple3<K, K, Edge<K, EV>>, Vertex<K, VV>, Tuple2<K, VV>> {
+
+		public void join(Tuple3<K, K, Edge<K, EV>> keysWithEdge, Vertex<K, VV> neighbor,
+				Collector<Tuple2<K, VV>> out) {
+
+			out.collect(new Tuple2<K, VV>(keysWithEdge.f0, neighbor.getValue()));
+		}
+	}
+
+	@ForwardedFieldsFirst("f0; f2->f1")
+	@ForwardedFieldsSecond("*->f2")
+	private static final class ProjectEdgeWithNeighbor<K, VV, EV> implements FlatJoinFunction<
+		Tuple3<K, K, Edge<K, EV>>, Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> {
+
+		public void join(Tuple3<K, K, Edge<K, EV>> keysWithEdge, Vertex<K, VV> neighbor,
+						Collector<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> out) {
+			out.collect(new Tuple3<K, Edge<K, EV>, Vertex<K, VV>>(keysWithEdge.f0, keysWithEdge.f2, neighbor));
+		}
+	}
+
+	private static final class ApplyNeighborCoGroupFunction<K, VV, EV, T> implements CoGroupFunction<
+		Vertex<K, VV>, Tuple2<Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
+
+		private NeighborsFunctionWithVertexValue<K, VV, EV, T> function;
+
+		public ApplyNeighborCoGroupFunction(NeighborsFunctionWithVertexValue<K, VV, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors,
+				Collector<T> out) throws Exception {
+			function.iterateNeighbors(vertex.iterator().next(),	neighbors, out);
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class,	function.getClass(), 3, null, null);
+		}
+	}
+
+	private static final class ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>
+			implements CoGroupFunction<Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
+
+		private NeighborsFunctionWithVertexValue<K, VV, EV, T> function;
+
+		public ApplyCoGroupFunctionOnAllNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> fun) {
+			this.function = fun;
+		}
+
+		public void coGroup(Iterable<Vertex<K, VV>> vertex,
+				final Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> keysWithNeighbors, 
+				Collector<T> out) throws Exception {
+
+			final Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighborsIterator = new Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>>() {
+
+				final Iterator<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> keysWithEdgesIterator = keysWithNeighbors.iterator();
+
+				@Override
+				public boolean hasNext() {
+					return keysWithEdgesIterator.hasNext();
+				}
+
+				@Override
+				public Tuple2<Edge<K, EV>, Vertex<K, VV>> next() {
+					Tuple3<K, Edge<K, EV>, Vertex<K, VV>> next = keysWithEdgesIterator.next();
+					return new Tuple2<Edge<K, EV>, Vertex<K, VV>>(next.f1, next.f2);
+				}
+
+				@Override
+				public void remove() {
+					keysWithEdgesIterator.remove();
+				}
+			};
+
+			Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighborsIterable = new Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>>() {
+				public Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>> iterator() {
+					return neighborsIterator;
+				}
+			};
+
+			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
+
+			if (vertexIterator.hasNext()) {
+				function.iterateNeighbors(vertexIterator.next(), neighborsIterable, out);
+			} else {
+				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
+			}
+		}
+
+		@Override
+		public TypeInformation<T> getProducedType() {
+			return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class,	function.getClass(), 3, null, null);
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the neighbor values of each
+	 * vertex.
+	 *
+	 * @param reduceNeighborsFunction the function to apply to the neighborhood
+	 * @param direction the edge direction (in-, out-, all-)
+	 * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
+	 * @throws IllegalArgumentException
+	 */
+	public DataSet<Tuple2<K, VV>> reduceOnNeighbors(ReduceNeighborsFunction<VV> reduceNeighborsFunction,
+									EdgeDirection direction) throws IllegalArgumentException {
+		switch (direction) {
+			case IN:
+				// create <vertex-source value> pairs
+				final DataSet<Tuple2<K, VV>> verticesWithSourceNeighborValues = edges
+						.join(this.vertices).where(0).equalTo(0)
+						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(1))
+						.withForwardedFieldsFirst("f1->f0");
+				return verticesWithSourceNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
+						reduceNeighborsFunction));
+			case OUT:
+				// create <vertex-target value> pairs
+				DataSet<Tuple2<K, VV>> verticesWithTargetNeighborValues = edges
+						.join(this.vertices).where(1).equalTo(0)
+						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(0))
+						.withForwardedFieldsFirst("f0");
+				return verticesWithTargetNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
+						reduceNeighborsFunction));
+			case ALL:
+				// create <vertex-neighbor value> pairs
+				DataSet<Tuple2<K, VV>> verticesWithNeighborValues = edges
+						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
+						.join(this.vertices).where(1).equalTo(0)
+						.with(new ProjectNeighborValue<K, VV, EV>());
+
+				return verticesWithNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
+						reduceNeighborsFunction));
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	@ForwardedFields("f0")
+	private static final class ApplyNeighborReduceFunction<K, VV> implements ReduceFunction<Tuple2<K, VV>> {
+
+		private ReduceNeighborsFunction<VV> function;
+
+		public ApplyNeighborReduceFunction(ReduceNeighborsFunction<VV> fun) {
+			this.function = fun;
+		}
+
+		@Override
+		public Tuple2<K, VV> reduce(Tuple2<K, VV> first, Tuple2<K, VV> second) throws Exception {
+			first.setField(function.reduceNeighbors(first.f1, second.f1), 1);
+			return first;
+		}
+	}
+
+	/**
+	 * Compute an aggregate over the edge values of each vertex.
+	 *
+	 * @param reduceEdgesFunction
+	 *            the function to apply to the neighborhood
+	 * @param direction
+	 *            the edge direction (in-, out-, all-)
+	 * @return a Dataset containing one value per vertex(vertex key, aggregate edge value)
+	 * @throws IllegalArgumentException
+	 */
+	public DataSet<Tuple2<K, EV>> reduceOnEdges(ReduceEdgesFunction<EV> reduceEdgesFunction,
+								EdgeDirection direction) throws IllegalArgumentException {
+
+		switch (direction) {
+			case IN:
+				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(1))
+						.withForwardedFields("f1->f0")
+						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
+			case OUT:
+				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(0))
+						.withForwardedFields("f0->f0")
+						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
+			case ALL:
+				return edges.flatMap(new EmitOneVertexWithEdgeValuePerNode<K, EV>())
+						.withForwardedFields("f2->f1")
+						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+	}
+
+	@ForwardedFields("f0")
+	private static final class ApplyReduceFunction<K, EV> implements ReduceFunction<Tuple2<K, EV>> {
+
+		private ReduceEdgesFunction<EV> function;
+
+		public ApplyReduceFunction(ReduceEdgesFunction<EV> fun) {
+			this.function = fun;
+		}
+
+		@Override
+		public Tuple2<K, EV> reduce(Tuple2<K, EV> first, Tuple2<K, EV> second) throws Exception {
+			first.setField(function.reduceEdges(first.f1, second.f1), 1);
+			return first;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
new file mode 100644
index 0000000..08cf011
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * @param <K> key type
+ * @param <VV> vertex value type
+ * @param <EV> edge value type
+ * @param <T> the return type
+ */
+public interface GraphAlgorithm<K, VV, EV, T> {
+
+	public T run(Graph<K, VV, EV> input) throws Exception;
+}


[18/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
new file mode 100755
index 0000000..23ccb68
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
@@ -0,0 +1,425 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
+import org.apache.flink.api.java.operators.CustomUnaryOperation;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.operators.JoinOperator;
+import org.apache.flink.api.java.operators.MapOperator;
+import org.apache.flink.api.java.operators.ReduceOperator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.util.Collector;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class represents iterative graph computations, programmed in a gather-sum-apply perspective.
+ *
+ * @param <K> The type of the vertex key in the graph
+ * @param <VV> The type of the vertex value in the graph
+ * @param <EV> The type of the edge value in the graph
+ * @param <M> The intermediate type used by the gather, sum and apply functions
+ */
+public class GatherSumApplyIteration<K, VV, EV, M> implements CustomUnaryOperation<Vertex<K, VV>,
+		Vertex<K, VV>> {
+
+	private DataSet<Vertex<K, VV>> vertexDataSet;
+	private DataSet<Edge<K, EV>> edgeDataSet;
+
+	private final GatherFunction<VV, EV, M> gather;
+	private final SumFunction<VV, EV, M> sum;
+	private final ApplyFunction<K, VV, M> apply;
+	private final int maximumNumberOfIterations;
+	private EdgeDirection direction = EdgeDirection.OUT;
+
+	private GSAConfiguration configuration;
+
+	// ----------------------------------------------------------------------------------
+
+	private GatherSumApplyIteration(GatherFunction<VV, EV, M> gather, SumFunction<VV, EV, M> sum,
+			ApplyFunction<K, VV, M> apply, DataSet<Edge<K, EV>> edges, int maximumNumberOfIterations) {
+
+		Preconditions.checkNotNull(gather);
+		Preconditions.checkNotNull(sum);
+		Preconditions.checkNotNull(apply);
+		Preconditions.checkNotNull(edges);
+		Preconditions.checkArgument(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one.");
+
+		this.gather = gather;
+		this.sum = sum;
+		this.apply = apply;
+		this.edgeDataSet = edges;
+		this.maximumNumberOfIterations = maximumNumberOfIterations;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Custom Operator behavior
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the input data set for this operator. In the case of this operator this input data set represents
+	 * the set of vertices with their initial state.
+	 *
+	 * @param dataSet The input data set, which in the case of this operator represents the set of
+	 *                vertices with their initial state.
+	 */
+	@Override
+	public void setInput(DataSet<Vertex<K, VV>> dataSet) {
+		this.vertexDataSet = dataSet;
+	}
+
+	/**
+	 * Computes the results of the gather-sum-apply iteration
+	 *
+	 * @return The resulting DataSet
+	 */
+	@Override
+	public DataSet<Vertex<K, VV>> createResult() {
+		if (vertexDataSet == null) {
+			throw new IllegalStateException("The input data set has not been set.");
+		}
+
+		// Prepare type information
+		TypeInformation<K> keyType = ((TupleTypeInfo<?>) vertexDataSet.getType()).getTypeAt(0);
+		TypeInformation<M> messageType = TypeExtractor.createTypeInfo(GatherFunction.class, gather.getClass(), 2, null, null);
+		TypeInformation<Tuple2<K, M>> innerType = new TupleTypeInfo<Tuple2<K, M>>(keyType, messageType);
+		TypeInformation<Vertex<K, VV>> outputType = vertexDataSet.getType();
+
+		// create a graph
+		Graph<K, VV, EV> graph =
+				Graph.fromDataSet(vertexDataSet, edgeDataSet, vertexDataSet.getExecutionEnvironment());
+
+		// check whether the numVertices option is set and, if so, compute the total number of vertices
+		// and set it within the gather, sum and apply functions
+		if (this.configuration != null && this.configuration.isOptNumVertices()) {
+			try {
+				long numberOfVertices = graph.numberOfVertices();
+				gather.setNumberOfVertices(numberOfVertices);
+				sum.setNumberOfVertices(numberOfVertices);
+				apply.setNumberOfVertices(numberOfVertices);
+			} catch (Exception e) {
+				e.printStackTrace();
+			}
+		}
+
+		// Prepare UDFs
+		GatherUdf<K, VV, EV, M> gatherUdf = new GatherUdf<K, VV, EV, M>(gather, innerType);
+		SumUdf<K, VV, EV, M> sumUdf = new SumUdf<K, VV, EV, M>(sum, innerType);
+		ApplyUdf<K, VV, EV, M> applyUdf = new ApplyUdf<K, VV, EV, M>(apply, outputType);
+
+		final int[] zeroKeyPos = new int[] {0};
+		final DeltaIteration<Vertex<K, VV>, Vertex<K, VV>> iteration =
+				vertexDataSet.iterateDelta(vertexDataSet, maximumNumberOfIterations, zeroKeyPos);
+
+		// set up the iteration operator
+		if (this.configuration != null) {
+
+			iteration.name(this.configuration.getName(
+					"Gather-sum-apply iteration (" + gather + " | " + sum + " | " + apply + ")"));
+			iteration.parallelism(this.configuration.getParallelism());
+			iteration.setSolutionSetUnManaged(this.configuration.isSolutionSetUnmanagedMemory());
+
+			// register all aggregators
+			for (Map.Entry<String, Aggregator<?>> entry : this.configuration.getAggregators().entrySet()) {
+				iteration.registerAggregator(entry.getKey(), entry.getValue());
+			}
+		}
+		else {
+			// no configuration provided; set default name
+			iteration.name("Gather-sum-apply iteration (" + gather + " | " + sum + " | " + apply + ")");
+		}
+
+		// Prepare the neighbors
+		if(this.configuration != null) {
+			direction = this.configuration.getDirection();
+		}
+		DataSet<Tuple2<K, Neighbor<VV, EV>>> neighbors;
+		switch(direction) {
+			case OUT:
+				neighbors = iteration
+				.getWorkset().join(edgeDataSet)
+				.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
+				break;
+			case IN:
+				neighbors = iteration
+				.getWorkset().join(edgeDataSet)
+				.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>());
+				break;
+			case ALL:
+				neighbors =  iteration
+						.getWorkset().join(edgeDataSet)
+						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>()).union(iteration
+								.getWorkset().join(edgeDataSet)
+								.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>()));
+				break;
+			default:
+				neighbors = iteration
+						.getWorkset().join(edgeDataSet)
+						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
+				break;
+		}
+
+		// Gather, sum and apply
+		MapOperator<Tuple2<K, Neighbor<VV, EV>>, Tuple2<K, M>> gatherMapOperator = neighbors.map(gatherUdf);
+
+		// configure map gather function with name and broadcast variables
+		gatherMapOperator = gatherMapOperator.name("Gather");
+
+		if (this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getGatherBcastVars()) {
+				gatherMapOperator = gatherMapOperator.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+		DataSet<Tuple2<K, M>> gatheredSet = gatherMapOperator;
+
+		ReduceOperator<Tuple2<K, M>> sumReduceOperator = gatheredSet.groupBy(0).reduce(sumUdf);
+
+		// configure reduce sum function with name and broadcast variables
+		sumReduceOperator = sumReduceOperator.name("Sum");
+
+		if (this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getSumBcastVars()) {
+				sumReduceOperator = sumReduceOperator.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+		DataSet<Tuple2<K, M>> summedSet = sumReduceOperator;
+
+		JoinOperator<?, ?, Vertex<K, VV>> appliedSet = summedSet
+				.join(iteration.getSolutionSet())
+				.where(0)
+				.equalTo(0)
+				.with(applyUdf);
+
+		// configure join apply function with name and broadcast variables
+		appliedSet = appliedSet.name("Apply");
+
+		if (this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getApplyBcastVars()) {
+				appliedSet = appliedSet.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+
+		// let the operator know that we preserve the key field
+		appliedSet.withForwardedFieldsFirst("0").withForwardedFieldsSecond("0");
+
+		return iteration.closeWith(appliedSet, appliedSet);
+	}
+
+	/**
+	 * Creates a new gather-sum-apply iteration operator for graphs
+	 *
+	 * @param edges The edge DataSet
+	 *
+	 * @param gather The gather function of the GSA iteration
+	 * @param sum The sum function of the GSA iteration
+	 * @param apply The apply function of the GSA iteration
+	 *
+	 * @param maximumNumberOfIterations The maximum number of iterations executed
+	 *
+	 * @param <K> The type of the vertex key in the graph
+	 * @param <VV> The type of the vertex value in the graph
+	 * @param <EV> The type of the edge value in the graph
+	 * @param <M> The intermediate type used by the gather, sum and apply functions
+	 *
+	 * @return An in stance of the gather-sum-apply graph computation operator.
+	 */
+	public static final <K, VV, EV, M> GatherSumApplyIteration<K, VV, EV, M>
+		withEdges(DataSet<Edge<K, EV>> edges, GatherFunction<VV, EV, M> gather,
+		SumFunction<VV, EV, M> sum, ApplyFunction<K, VV, M> apply, int maximumNumberOfIterations) {
+
+		return new GatherSumApplyIteration<K, VV, EV, M>(gather, sum, apply, edges, maximumNumberOfIterations);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Wrapping UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	@ForwardedFields("f0")
+	private static final class GatherUdf<K, VV, EV, M> extends RichMapFunction<Tuple2<K, Neighbor<VV, EV>>,
+			Tuple2<K, M>> implements ResultTypeQueryable<Tuple2<K, M>> {
+
+		private final GatherFunction<VV, EV, M> gatherFunction;
+		private transient TypeInformation<Tuple2<K, M>> resultType;
+
+		private GatherUdf(GatherFunction<VV, EV, M> gatherFunction, TypeInformation<Tuple2<K, M>> resultType) {
+			this.gatherFunction = gatherFunction;
+			this.resultType = resultType;
+		}
+
+		@Override
+		public Tuple2<K, M> map(Tuple2<K, Neighbor<VV, EV>> neighborTuple) {
+			M result = this.gatherFunction.gather(neighborTuple.f1);
+			return new Tuple2<K, M>(neighborTuple.f0, result);
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
+				this.gatherFunction.init(getIterationRuntimeContext());
+			}
+			this.gatherFunction.preSuperstep();
+		}
+
+		@Override
+		public void close() throws Exception {
+			this.gatherFunction.postSuperstep();
+		}
+
+		@Override
+		public TypeInformation<Tuple2<K, M>> getProducedType() {
+			return this.resultType;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumUdf<K, VV, EV, M> extends RichReduceFunction<Tuple2<K, M>>
+			implements ResultTypeQueryable<Tuple2<K, M>>{
+
+		private final SumFunction<VV, EV, M> sumFunction;
+		private transient TypeInformation<Tuple2<K, M>> resultType;
+
+		private SumUdf(SumFunction<VV, EV, M> sumFunction, TypeInformation<Tuple2<K, M>> resultType) {
+			this.sumFunction = sumFunction;
+			this.resultType = resultType;
+		}
+
+		@Override
+		public Tuple2<K, M> reduce(Tuple2<K, M> arg0, Tuple2<K, M> arg1) throws Exception {
+			K key = arg0.f0;
+			M result = this.sumFunction.sum(arg0.f1, arg1.f1);
+			return new Tuple2<K, M>(key, result);
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
+				this.sumFunction.init(getIterationRuntimeContext());
+			}
+			this.sumFunction.preSuperstep();
+		}
+
+		@Override
+		public void close() throws Exception {
+			this.sumFunction.postSuperstep();
+		}
+
+		@Override
+		public TypeInformation<Tuple2<K, M>> getProducedType() {
+			return this.resultType;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ApplyUdf<K, VV, EV, M> extends RichFlatJoinFunction<Tuple2<K, M>,
+			Vertex<K, VV>, Vertex<K, VV>> implements ResultTypeQueryable<Vertex<K, VV>> {
+
+		private final ApplyFunction<K, VV, M> applyFunction;
+		private transient TypeInformation<Vertex<K, VV>> resultType;
+
+		private ApplyUdf(ApplyFunction<K, VV, M> applyFunction, TypeInformation<Vertex<K, VV>> resultType) {
+			this.applyFunction = applyFunction;
+			this.resultType = resultType;
+		}
+
+		@Override
+		public void join(Tuple2<K, M> newValue, final Vertex<K, VV> currentValue, final Collector<Vertex<K, VV>> out) throws Exception {
+
+			this.applyFunction.setOutput(currentValue, out);
+			this.applyFunction.apply(newValue.f1, currentValue.getValue());
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
+				this.applyFunction.init(getIterationRuntimeContext());
+			}
+			this.applyFunction.preSuperstep();
+		}
+
+		@Override
+		public void close() throws Exception {
+			this.applyFunction.postSuperstep();
+		}
+
+		@Override
+		public TypeInformation<Vertex<K, VV>> getProducedType() {
+			return this.resultType;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	@ForwardedFieldsSecond("f1->f0")
+	private static final class ProjectKeyWithNeighborOUT<K, VV, EV> implements FlatJoinFunction<
+			Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Neighbor<VV, EV>>> {
+
+		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple2<K, Neighbor<VV, EV>>> out) {
+			out.collect(new Tuple2<K, Neighbor<VV, EV>>(
+					edge.getTarget(), new Neighbor<VV, EV>(vertex.getValue(), edge.getValue())));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	@ForwardedFieldsSecond({"f0"})
+	private static final class ProjectKeyWithNeighborIN<K, VV, EV> implements FlatJoinFunction<
+			Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Neighbor<VV, EV>>> {
+
+		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple2<K, Neighbor<VV, EV>>> out) {
+			out.collect(new Tuple2<K, Neighbor<VV, EV>>(
+					edge.getSource(), new Neighbor<VV, EV>(vertex.getValue(), edge.getValue())));
+		}
+	}
+
+
+
+
+	/**
+	 * Configures this gather-sum-apply iteration with the provided parameters.
+	 *
+	 * @param parameters the configuration parameters
+	 */
+	public void configure(GSAConfiguration parameters) {
+		this.configuration = parameters;
+	}
+
+	/**
+	 * @return the configuration parameters of this gather-sum-apply iteration
+	 */
+	public GSAConfiguration getIterationConfiguration() {
+		return this.configuration;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
new file mode 100755
index 0000000..7fa1ed2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
@@ -0,0 +1,45 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/**
+ * This class represents a <sourceVertex, edge> pair
+ * This is a wrapper around Tuple2<VV, EV> for convenience in the GatherFunction
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ */
+@SuppressWarnings("serial")
+public class Neighbor<VV, EV> extends Tuple2<VV, EV> {
+
+	public Neighbor() {}
+
+	public Neighbor(VV neighborValue, EV edgeValue) {
+		super(neighborValue, edgeValue);
+	}
+
+	public VV getNeighborValue() {
+		return this.f0;
+	}
+
+	public EV getEdgeValue() {
+		return this.f1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
new file mode 100755
index 0000000..f27e275
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
@@ -0,0 +1,134 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.types.Value;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+/**
+ * The base class for the second step of a {@link GatherSumApplyIteration}.
+ *
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ * @param <M> the output type
+ */
+@SuppressWarnings("serial")
+public abstract class SumFunction<VV, EV, M> implements Serializable {
+
+	// --------------------------------------------------------------------------------------------
+	//  Attribute that allows access to the total number of vertices inside an iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private long numberOfVertices = -1L;
+
+	/**
+	 * Retrieves the number of vertices in the graph.
+	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * option has been set; -1 otherwise.
+	 */
+	public long getNumberOfVertices() {
+		return numberOfVertices;
+	}
+
+	void setNumberOfVertices(long numberOfVertices) {
+		this.numberOfVertices = numberOfVertices;
+	}
+
+	//---------------------------------------------------------------------------------------------
+	/**
+	 * This method is invoked once per superstep, after the {@link GatherFunction} 
+	 * in a {@link GatherSumApplyIteration}.
+	 * It combines the partial values produced by {@link GatherFunction#gather(Neighbor)}
+	 * in pairs, until a single value has been computed.
+	 * 
+	 * @param arg0 the first partial value.
+	 * @param arg1 the second partial value.
+	 * @return the combined value.
+	 */
+	public abstract M sum(M arg0, M arg1);
+
+	/**
+	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
+	 */
+	public void preSuperstep() {}
+
+	/**
+	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
+	 */
+	public void postSuperstep() {}
+
+	/**
+	 * Gets the number of the superstep, starting at <tt>1</tt>.
+	 *
+	 * @return The number of the current superstep.
+	 */
+	public int getSuperstepNumber() {
+		return this.runtimeContext.getSuperstepNumber();
+	}
+
+	/**
+	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
+	 * all aggregates globally once per superstep and makes them available in the next superstep.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
+	 */
+	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+		return this.runtimeContext.<T>getIterationAggregator(name);
+	}
+
+	/**
+	 * Get the aggregated value that an aggregator computed in the previous iteration.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregated value of the previous iteration.
+	 */
+	public <T extends Value> T getPreviousIterationAggregate(String name) {
+		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
+	}
+
+	/**
+	 * Gets the broadcast data set registered under the given name. Broadcast data sets
+	 * are available on all parallel instances of a function.
+	 *
+	 * @param name The name under which the broadcast set is registered.
+	 * @return The broadcast data set.
+	 */
+	public <T> Collection<T> getBroadcastSet(String name) {
+		return this.runtimeContext.<T>getBroadcastVariable(name);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal methods
+	// --------------------------------------------------------------------------------------------
+
+	private IterationRuntimeContext runtimeContext;
+
+	public void init(IterationRuntimeContext iterationRuntimeContext) {
+		this.runtimeContext = iterationRuntimeContext;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
new file mode 100644
index 0000000..0dd39fc
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
@@ -0,0 +1,183 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+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.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * Community Detection Algorithm.
+ *
+ * This implementation expects Long Vertex values and labels. The Vertex values of the input Graph provide the initial label assignments.
+ * 
+ * Initially, each vertex is assigned a tuple formed of its own initial value along with a score equal to 1.0.
+ * The vertices propagate their labels and max scores in iterations, each time adopting the label with the
+ * highest score from the list of received messages. The chosen label is afterwards re-scored using the fraction
+ * delta/the superstep number. Delta is passed as a parameter and has 0.5 as a default value.
+ *
+ * The algorithm converges when vertices no longer update their value or when the maximum number of iterations
+ * is reached.
+ * 
+ * @param <K> the Vertex ID type 
+ *
+ * @see <a href="http://arxiv.org/pdf/0808.2633.pdf">article explaining the algorithm in detail</a>
+ */
+public class CommunityDetection<K> implements GraphAlgorithm<K, Long, Double, Graph<K, Long, Double>> {
+
+	private Integer maxIterations;
+
+	private Double delta;
+
+	public CommunityDetection(Integer maxIterations, Double delta) {
+
+		this.maxIterations = maxIterations;
+		this.delta = delta;
+	}
+
+	@Override
+	public Graph<K, Long, Double> run(Graph<K, Long, Double> graph) {
+
+		DataSet<Vertex<K, Tuple2<Long, Double>>> initializedVertices = graph.getVertices()
+				.map(new AddScoreToVertexValuesMapper<K>());
+
+		Graph<K, Tuple2<Long, Double>, Double> graphWithScoredVertices =
+				Graph.fromDataSet(initializedVertices, graph.getEdges(), graph.getContext()).getUndirected();
+
+		return graphWithScoredVertices.runVertexCentricIteration(new VertexLabelUpdater<K>(delta),
+				new LabelMessenger<K>(), maxIterations)
+				.mapVertices(new RemoveScoreFromVertexValuesMapper<K>());
+	}
+
+	@SuppressWarnings("serial")
+	public static final class VertexLabelUpdater<K> extends VertexUpdateFunction<
+		K, Tuple2<Long, Double>, Tuple2<Long, Double>> {
+
+		private Double delta;
+
+		public VertexLabelUpdater(Double delta) {
+			this.delta = delta;
+		}
+
+		@Override
+		public void updateVertex(Vertex<K, Tuple2<Long, Double>> vertex,
+								MessageIterator<Tuple2<Long, Double>> inMessages) throws Exception {
+
+			// we would like these two maps to be ordered
+			Map<Long, Double> receivedLabelsWithScores = new TreeMap<Long, Double>();
+			Map<Long, Double> labelsWithHighestScore = new TreeMap<Long, Double>();
+
+			for (Tuple2<Long, Double> message : inMessages) {
+				// split the message into received label and score
+				Long receivedLabel = message.f0;
+				Double receivedScore = message.f1;
+
+				// if the label was received before
+				if (receivedLabelsWithScores.containsKey(receivedLabel)) {
+					Double newScore = receivedScore + receivedLabelsWithScores.get(receivedLabel);
+					receivedLabelsWithScores.put(receivedLabel, newScore);
+				} else {
+					// first time we see the label
+					receivedLabelsWithScores.put(receivedLabel, receivedScore);
+				}
+
+				// store the labels with the highest scores
+				if (labelsWithHighestScore.containsKey(receivedLabel)) {
+					Double currentScore = labelsWithHighestScore.get(receivedLabel);
+					if (currentScore < receivedScore) {
+						// record the highest score
+						labelsWithHighestScore.put(receivedLabel, receivedScore);
+					}
+				} else {
+					// first time we see this label
+					labelsWithHighestScore.put(receivedLabel, receivedScore);
+				}
+			}
+
+			if(receivedLabelsWithScores.size() > 0) {
+				// find the label with the highest score from the ones received
+				Double maxScore = -Double.MAX_VALUE;
+				Long maxScoreLabel = vertex.getValue().f0;
+				for (Long curLabel : receivedLabelsWithScores.keySet()) {
+
+					if (receivedLabelsWithScores.get(curLabel) > maxScore) {
+						maxScore = receivedLabelsWithScores.get(curLabel);
+						maxScoreLabel = curLabel;
+					}
+				}
+
+				// find the highest score of maxScoreLabel
+				Double highestScore = labelsWithHighestScore.get(maxScoreLabel);
+				// re-score the new label
+				if (maxScoreLabel != vertex.getValue().f0) {
+					highestScore -= delta / getSuperstepNumber();
+				}
+				// else delta = 0
+				// update own label
+				setNewVertexValue(new Tuple2<Long, Double>(maxScoreLabel, highestScore));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class LabelMessenger<K> extends MessagingFunction<K, Tuple2<Long, Double>,
+			Tuple2<Long, Double>, Double> {
+
+		@Override
+		public void sendMessages(Vertex<K, Tuple2<Long, Double>> vertex) throws Exception {
+
+			for(Edge<K, Double> edge : getEdges()) {
+				sendMessageTo(edge.getTarget(), new Tuple2<Long, Double>(vertex.getValue().f0,
+						vertex.getValue().f1 * edge.getValue()));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	@ForwardedFields("f0")
+	public static final class AddScoreToVertexValuesMapper<K> implements MapFunction<
+		Vertex<K, Long>, Vertex<K, Tuple2<Long, Double>>> {
+
+		public Vertex<K, Tuple2<Long, Double>> map(Vertex<K, Long> vertex) {
+			return new Vertex<K, Tuple2<Long, Double>>(
+					vertex.getId(), new Tuple2<Long, Double>(vertex.getValue(), 1.0));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static final class RemoveScoreFromVertexValuesMapper<K> implements MapFunction<
+		Vertex<K, Tuple2<Long, Double>>, Long> {
+
+		@Override
+		public Long map(Vertex<K, Tuple2<Long, Double>> vertex) throws Exception {
+			return vertex.getValue().f0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/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
new file mode 100644
index 0000000..ed853fe
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.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.library;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.types.NullValue;
+
+/**
+ * A vertex-centric implementation of the Connected Components algorithm.
+ *
+ * This implementation assumes that the vertices of the input Graph are initialized with unique, Long component IDs.
+ * The vertices propagate their current component ID in iterations, each time adopting a new value from the received neighbor IDs,
+ * provided that the value is less than the current minimum.
+ *
+ * The algorithm converges when vertices no longer update their value or when the maximum number of iterations
+ * is reached.
+ * 
+ * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
+ * 
+ * @see {@link org.apache.flink.graph.library.GSAConnectedComponents}
+ */
+@SuppressWarnings("serial")
+public class ConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {
+
+	private Integer maxIterations;
+
+	public ConnectedComponents(Integer maxIterations) {
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> graph) throws Exception {
+
+		Graph<K, Long, NullValue> undirectedGraph = graph.mapEdges(new NullValueEdgeMapper<K, EV>())
+				.getUndirected();
+
+		// initialize vertex values and run the Vertex Centric Iteration
+		return undirectedGraph.runVertexCentricIteration(
+				new CCUpdater<K>(), new CCMessenger<K>(), maxIterations)
+				.getVertices();
+	}
+
+	/**
+	 * Updates the value of a vertex by picking the minimum neighbor ID out of all the incoming messages.
+	 */
+	public static final class CCUpdater<K> extends VertexUpdateFunction<K, Long, Long> {
+
+		@Override
+		public void updateVertex(Vertex<K, Long> vertex, MessageIterator<Long> messages) throws Exception {
+			long min = Long.MAX_VALUE;
+
+			for (long msg : messages) {
+				min = Math.min(min, msg);
+			}
+
+			// update vertex value, if new minimum
+			if (min < vertex.getValue()) {
+				setNewVertexValue(min);
+			}
+		}
+	}
+
+	/**
+	 * Distributes the minimum ID associated with a given vertex among all the target vertices.
+	 */
+	public static final class CCMessenger<K> extends MessagingFunction<K, Long, Long, NullValue> {
+
+		@Override
+		public void sendMessages(Vertex<K, Long> vertex) throws Exception {
+			// send current minimum to neighbors
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/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
new file mode 100755
index 0000000..77bc2cf
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
@@ -0,0 +1,88 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.gsa.ApplyFunction;
+import org.apache.flink.graph.gsa.GatherFunction;
+import org.apache.flink.graph.gsa.SumFunction;
+import org.apache.flink.graph.gsa.Neighbor;
+import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.types.NullValue;
+
+/**
+ * This is an implementation of the Connected Components algorithm, using a gather-sum-apply iteration.
+ * This implementation assumes that the vertices of the input Graph are initialized with unique, Long component IDs.
+ * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
+ * 
+ * @see {@link org.apache.flink.graph.library.ConnectedComponents}
+ */
+public class GSAConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {
+
+	private Integer maxIterations;
+
+	public GSAConnectedComponents(Integer maxIterations) {
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> graph) throws Exception {
+
+		Graph<K, Long, NullValue> undirectedGraph = graph.mapEdges(new NullValueEdgeMapper<K, EV>())
+				.getUndirected();
+
+		// initialize vertex values and run the Vertex Centric Iteration
+		return undirectedGraph.runGatherSumApplyIteration(
+				new GatherNeighborIds(), new SelectMinId(), new UpdateComponentId<K>(),
+				maxIterations).getVertices();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Connected Components UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
+
+		public Long gather(Neighbor<Long, NullValue> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
+
+		public Long sum(Long newValue, Long currentValue) {
+			return Math.min(newValue, currentValue);
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class UpdateComponentId<K> extends ApplyFunction<K, Long, Long> {
+
+		public void apply(Long summedValue, Long origValue) {
+			if (summedValue < origValue) {
+				setResult(summedValue);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
new file mode 100644
index 0000000..df3e89a
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
@@ -0,0 +1,135 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+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;
+
+/**
+ * This is an implementation of a simple PageRank algorithm, using a gather-sum-apply iteration.
+ * The user can define the damping factor and the maximum number of iterations.
+ * If the number of vertices of the input graph is known, it should be provided as a parameter
+ * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices.
+ * 
+ * The implementation assumes that each page has at least one incoming and one outgoing link.
+ */
+public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
+
+	private double beta;
+	private int maxIterations;
+	private long numberOfVertices;
+
+	/**
+	 * @param beta the damping factor
+	 * @param maxIterations the maximum number of iterations
+	 */
+	public GSAPageRank(double beta, int maxIterations) {
+		this.beta = beta;
+		this.maxIterations = maxIterations;
+	}
+
+	public GSAPageRank(double beta, long numVertices, int maxIterations) {
+		this.beta = beta;
+		this.numberOfVertices = numVertices;
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> network) throws Exception {
+
+		if (numberOfVertices == 0) {
+			numberOfVertices = network.numberOfVertices();
+		}
+
+		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
+
+		Graph<K, Double, Double> networkWithWeights = network
+				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
+
+		return networkWithWeights.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(),
+				new UpdateRanks<K>(beta, numberOfVertices), maxIterations)
+				.getVertices();
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Page Rank UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static final class GatherRanks extends GatherFunction<Double, Double, Double> {
+
+		long numberOfVertices;
+
+		public GatherRanks(long numberOfVertices) {
+			this.numberOfVertices = numberOfVertices;
+		}
+
+		@Override
+		public Double gather(Neighbor<Double, Double> neighbor) {
+			double neighborRank = neighbor.getNeighborValue();
+
+			if(getSuperstepNumber() == 1) {
+				neighborRank = 1.0 / numberOfVertices;
+			}
+
+			return neighborRank * neighbor.getEdgeValue();
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumRanks extends SumFunction<Double, Double, Double> {
+
+		@Override
+		public Double sum(Double newValue, Double currentValue) {
+			return newValue + currentValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class UpdateRanks<K> extends ApplyFunction<K, Double, Double> {
+
+		private final double beta;
+		private final long numVertices;
+
+		public UpdateRanks(double beta, long numberOfVertices) {
+			this.beta = beta;
+			this.numVertices = numberOfVertices;
+		}
+
+		@Override
+		public void apply(Double rankSum, Double currentValue) {
+			setResult((1-beta)/numVertices + beta * rankSum);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
+		public Double map(Tuple2<Double, Long> value) {
+			return value.f0 / value.f1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
new file mode 100755
index 0000000..5a76072
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
@@ -0,0 +1,101 @@
+/*
+ * 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.library;
+
+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.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.gsa.ApplyFunction;
+import org.apache.flink.graph.gsa.GatherFunction;
+import org.apache.flink.graph.gsa.SumFunction;
+import org.apache.flink.graph.gsa.Neighbor;
+
+/**
+ * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration
+ */
+public class GSASingleSourceShortestPaths<K> implements
+	GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
+
+	private final K srcVertexId;
+	private final Integer maxIterations;
+
+	public GSASingleSourceShortestPaths(K srcVertexId, Integer maxIterations) {
+		this.srcVertexId = srcVertexId;
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> input) {
+
+		return input.mapVertices(new InitVerticesMapper<K>(srcVertexId))
+				.runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(),
+						new UpdateDistance<K>(), maxIterations)
+						.getVertices();
+	}
+
+	@SuppressWarnings("serial")
+	public static final class InitVerticesMapper<K>	implements MapFunction<Vertex<K, Double>, Double> {
+
+		private K srcVertexId;
+
+		public InitVerticesMapper(K srcId) {
+			this.srcVertexId = srcId;
+		}
+
+		public Double map(Vertex<K, Double> value) {
+			if (value.f0.equals(srcVertexId)) {
+				return 0.0;
+			} else {
+				return Double.MAX_VALUE;
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Single Source Shortest Path UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static final class CalculateDistances extends GatherFunction<Double, Double, Double> {
+
+		public Double gather(Neighbor<Double, Double> neighbor) {
+			return neighbor.getNeighborValue() + neighbor.getEdgeValue();
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class ChooseMinDistance extends SumFunction<Double, Double, Double> {
+
+		public Double sum(Double newValue, Double currentValue) {
+			return Math.min(newValue, currentValue);
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class UpdateDistance<K> extends ApplyFunction<K, Double, Double> {
+
+		public void apply(Double newDistance, Double oldDistance) {
+			if (newDistance < oldDistance) {
+				setResult(newDistance);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
new file mode 100644
index 0000000..76d170d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
@@ -0,0 +1,190 @@
+/*
+ * 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.library;
+
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+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.GraphAlgorithm;
+import org.apache.flink.graph.ReduceNeighborsFunction;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Triplet;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.types.NullValue;
+
+import java.util.TreeMap;
+
+/**
+ * Triangle Count Algorithm.
+ *
+ * This algorithm operates in three phases. First, vertices select neighbors with id greater than theirs
+ * and send messages to them. Each received message is then propagated to neighbors with higher id.
+ * Finally, if a node encounters the target id in the list of received messages, it increments the number
+ * of triangles found.
+ *
+ * This implementation is non - iterative.
+ *
+ * The algorithm takes an undirected, unweighted graph as input and outputs a DataSet
+ * which contains a single integer representing the number of triangles.
+ */
+public class GSATriangleCount<K extends Comparable<K>, VV, EV> implements
+		GraphAlgorithm<K, VV, EV, DataSet<Integer>> {
+
+	@SuppressWarnings("serial")
+	@Override
+	public DataSet<Integer> run(Graph<K, VV, EV> input) throws Exception {
+
+		ExecutionEnvironment env = input.getContext();
+
+		// order the edges so that src is always higher than trg
+		DataSet<Edge<K, NullValue>> edges = input.getEdges().map(new OrderEdges<K, EV>()).distinct();
+
+		Graph<K, TreeMap<K, Integer>, NullValue> graph = Graph.fromDataSet(edges,
+				new VertexInitializer<K>(), env);
+
+		// select neighbors with ids higher than the current vertex id
+		// Gather: a no-op in this case
+		// Sum: create the set of neighbors
+		DataSet<Tuple2<K, TreeMap<K, Integer>>> higherIdNeighbors =
+				graph.reduceOnNeighbors(new GatherHigherIdNeighbors<K>(), EdgeDirection.IN);
+
+		Graph<K, TreeMap<K, Integer>, NullValue> graphWithReinitializedVertexValues =
+				graph.mapVertices(new VertexInitializerEmptyTreeMap<K>());
+
+		// Apply: attach the computed values to the vertices
+		// joinWithVertices to update the node values
+		DataSet<Vertex<K, TreeMap<K, Integer>>> verticesWithHigherIdNeighbors =
+				graphWithReinitializedVertexValues.joinWithVertices(higherIdNeighbors, new AttachValues<K>()).getVertices();
+
+		Graph<K, TreeMap<K,Integer>, NullValue> graphWithNeighbors = Graph.fromDataSet(verticesWithHigherIdNeighbors,
+				edges, env);
+
+		// propagate each received value to neighbors with higher id
+		// Gather: a no-op in this case
+		// Sum: propagate values
+		DataSet<Tuple2<K, TreeMap<K, Integer>>> propagatedValues = graphWithNeighbors
+				.reduceOnNeighbors(new GatherHigherIdNeighbors<K>(), EdgeDirection.IN);
+
+		// Apply: attach propagated values to vertices
+		DataSet<Vertex<K, TreeMap<K, Integer>>> verticesWithPropagatedValues =
+				graphWithReinitializedVertexValues.joinWithVertices(propagatedValues, new AttachValues<K>()).getVertices();
+
+		Graph<K, TreeMap<K, Integer>, NullValue> graphWithPropagatedNeighbors =
+				Graph.fromDataSet(verticesWithPropagatedValues, graphWithNeighbors.getEdges(), env);
+
+		// Scatter: compute the number of triangles
+		DataSet<Integer> numberOfTriangles = graphWithPropagatedNeighbors.getTriplets()
+				.map(new ComputeTriangles<K>()).reduce(new ReduceFunction<Integer>() {
+
+					@Override
+					public Integer reduce(Integer first, Integer second) throws Exception {
+						return first + second;
+					}
+				});
+
+		return numberOfTriangles;
+	}
+
+	@SuppressWarnings("serial")
+	private static final class OrderEdges<K extends Comparable<K>, EV> implements
+		MapFunction<Edge<K, EV>, Edge<K, NullValue>> {
+
+		@Override
+		public Edge<K, NullValue> map(Edge<K, EV> edge) throws Exception {
+			if (edge.getSource().compareTo(edge.getTarget()) < 0) {
+				return new Edge<K, NullValue>(edge.getTarget(), edge.getSource(), NullValue.getInstance());
+			} else {
+				return new Edge<K, NullValue>(edge.getSource(), edge.getTarget(), NullValue.getInstance());
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class VertexInitializer<K> implements MapFunction<K, TreeMap<K, Integer>> {
+
+		@Override
+		public TreeMap<K, Integer> map(K value) throws Exception {
+			TreeMap<K, Integer> neighbors = new TreeMap<K, Integer>();
+			neighbors.put(value, 1);
+
+			return neighbors;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class VertexInitializerEmptyTreeMap<K> implements
+			MapFunction<Vertex<K, TreeMap<K, Integer>>, TreeMap<K, Integer>> {
+
+		@Override
+		public TreeMap<K, Integer> map(Vertex<K, TreeMap<K, Integer>> vertex) throws Exception {
+			return new TreeMap<K, Integer>();
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AttachValues<K> implements MapFunction<Tuple2<TreeMap<K, Integer>,
+			TreeMap<K, Integer>>, TreeMap<K, Integer>> {
+
+		@Override
+		public TreeMap<K, Integer> map(Tuple2<TreeMap<K, Integer>, TreeMap<K, Integer>> tuple2) throws Exception {
+			return tuple2.f1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class GatherHigherIdNeighbors<K> implements
+		ReduceNeighborsFunction<TreeMap<K,Integer>> {
+
+		@Override
+		public TreeMap<K, Integer> reduceNeighbors(TreeMap<K,Integer> first, TreeMap<K,Integer> second) {
+			for (K key : second.keySet()) {
+				Integer value = first.get(key);
+				if (value != null) {
+					first.put(key, value + second.get(key));
+				} else {
+					first.put(key, second.get(key));
+				}
+			}
+			return first;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ComputeTriangles<K> implements MapFunction<Triplet<K, TreeMap<K, Integer>, NullValue>,
+			Integer> {
+
+		@Override
+		public Integer map(Triplet<K, TreeMap<K, Integer>, NullValue> triplet) throws Exception {
+
+			Vertex<K, TreeMap<K, Integer>> srcVertex = triplet.getSrcVertex();
+			Vertex<K, TreeMap<K, Integer>> trgVertex = triplet.getTrgVertex();
+			int triangles = 0;
+
+			if(trgVertex.getValue().get(srcVertex.getId()) != null) {
+				triangles = trgVertex.getValue().get(srcVertex.getId());
+			}
+			return triangles;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/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
new file mode 100644
index 0000000..82dfee7
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
@@ -0,0 +1,116 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.graph.utils.NullValueEdgeMapper;
+import org.apache.flink.types.NullValue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An implementation of the label propagation algorithm. The iterative algorithm
+ * detects communities by propagating labels. In each iteration, a vertex adopts
+ * the label that is most frequent among its neighbors' labels. Labels are
+ * represented by Longs and we assume a total ordering among them, in order to
+ * break ties. The algorithm converges when no vertex changes its value or the
+ * maximum number of iterations have been reached. Note that different
+ * initializations might lead to different results.
+ * 
+ */
+@SuppressWarnings("serial")
+
+public class LabelPropagation<K extends Comparable<K>, EV> implements GraphAlgorithm<K, Long, EV,
+	DataSet<Vertex<K, Long>>> {
+
+	private final int maxIterations;
+
+	public LabelPropagation(int maxIterations) {
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> input) {
+
+		// iteratively adopt the most frequent label among the neighbors
+		// of each vertex
+		return input.mapEdges(new NullValueEdgeMapper<K, EV>()).runVertexCentricIteration(new UpdateVertexLabel<K>(), new SendNewLabelToNeighbors<K>(),
+				maxIterations).getVertices();
+	}
+
+	/**
+	 * Function that updates the value of a vertex by adopting the most frequent
+	 * label among its in-neighbors
+	 */
+	public static final class UpdateVertexLabel<K> extends VertexUpdateFunction<K, Long, Long> {
+
+		public void updateVertex(Vertex<K, Long> vertex,
+				MessageIterator<Long> inMessages) {
+			Map<Long, Long> labelsWithFrequencies = new HashMap<Long, Long>();
+
+			long maxFrequency = 1;
+			long mostFrequentLabel = vertex.getValue();
+
+			// store the labels with their frequencies
+			for (Long msg : inMessages) {
+				if (labelsWithFrequencies.containsKey(msg)) {
+					long currentFreq = labelsWithFrequencies.get(msg);
+					labelsWithFrequencies.put(msg, currentFreq + 1);
+				} else {
+					labelsWithFrequencies.put(msg, 1L);
+				}
+			}
+			// select the most frequent label: if two or more labels have the
+			// same frequency,
+			// the node adopts the label with the highest value
+			for (Entry<Long, Long> entry : labelsWithFrequencies.entrySet()) {
+				if (entry.getValue() == maxFrequency) {
+					// check the label value to break ties
+					if (entry.getKey() > mostFrequentLabel) {
+						mostFrequentLabel = entry.getKey();
+					}
+				} else if (entry.getValue() > maxFrequency) {
+					maxFrequency = entry.getValue();
+					mostFrequentLabel = entry.getKey();
+				}
+			}
+
+			// set the new vertex value
+			setNewVertexValue(mostFrequentLabel);
+		}
+	}
+
+	/**
+	 * Sends the vertex label to all out-neighbors
+	 */
+	public static final class SendNewLabelToNeighbors<K> extends MessagingFunction<K, Long, Long, NullValue> {
+
+		public void sendMessages(Vertex<K, Long> vertex) {
+			sendMessageToAllNeighbors(vertex.getValue());
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
new file mode 100644
index 0000000..8193dba
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
@@ -0,0 +1,146 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+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.GraphAlgorithm;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+
+/**
+ * This is an implementation of a simple PageRank algorithm, using a vertex-centric iteration.
+ * The user can define the damping factor and the maximum number of iterations.
+ * If the number of vertices of the input graph is known, it should be provided as a parameter
+ * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices.
+ * 
+ * The implementation assumes that each page has at least one incoming and one outgoing link.
+ */
+public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
+
+	private double beta;
+	private int maxIterations;
+	private long numberOfVertices;
+
+	/**
+	 * @param beta the damping factor
+	 * @param maxIterations the maximum number of iterations
+	 */
+	public PageRank(double beta, int maxIterations) {
+		this.beta = beta;
+		this.maxIterations = maxIterations;
+		this.numberOfVertices = 0;
+	}
+
+	/**
+	 * @param beta the damping factor
+	 * @param maxIterations the maximum number of iterations
+	 * @param numVertices the number of vertices in the input
+	 */
+	public PageRank(double beta, long numVertices, int maxIterations) {
+		this.beta = beta;
+		this.maxIterations = maxIterations;
+		this.numberOfVertices = numVertices;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> network) throws Exception {
+
+		if (numberOfVertices == 0) {
+			numberOfVertices = network.numberOfVertices();
+		}
+
+		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
+
+		Graph<K, Double, Double> networkWithWeights = network
+				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
+
+		return networkWithWeights.runVertexCentricIteration(new VertexRankUpdater<K>(beta, numberOfVertices),
+				new RankMessenger<K>(numberOfVertices), maxIterations)
+				.getVertices();
+	}
+
+	/**
+	 * Function that updates the rank of a vertex by summing up the partial
+	 * ranks from all incoming messages and then applying the dampening formula.
+	 */
+	@SuppressWarnings("serial")
+	public static final class VertexRankUpdater<K> extends VertexUpdateFunction<K, Double, Double> {
+
+		private final double beta;
+		private final long numVertices;
+		
+		public VertexRankUpdater(double beta, long numberOfVertices) {
+			this.beta = beta;
+			this.numVertices = numberOfVertices;
+		}
+
+		@Override
+		public void updateVertex(Vertex<K, Double> vertex, MessageIterator<Double> inMessages) {
+			double rankSum = 0.0;
+			for (double msg : inMessages) {
+				rankSum += msg;
+			}
+
+			// apply the dampening factor / random jump
+			double newRank = (beta * rankSum) + (1 - beta) / numVertices;
+			setNewVertexValue(newRank);
+		}
+	}
+
+	/**
+	 * Distributes the rank of a vertex among all target vertices according to
+	 * the transition probability, which is associated with an edge as the edge
+	 * value.
+	 */
+	@SuppressWarnings("serial")
+	public static final class RankMessenger<K> extends MessagingFunction<K, Double, Double, Double> {
+
+		private final long numVertices;
+
+		public RankMessenger(long numberOfVertices) {
+			this.numVertices = numberOfVertices;
+		}
+
+		@Override
+		public void sendMessages(Vertex<K, Double> vertex) {
+			if (getSuperstepNumber() == 1) {
+				// initialize vertex ranks
+				vertex.setValue(new Double(1.0 / numVertices));
+			}
+
+			for (Edge<K, Double> edge : getEdges()) {
+				sendMessageTo(edge.getTarget(), vertex.getValue() * edge.getValue());
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
+		public Double map(Tuple2<Double, Long> value) {
+			return value.f0 / value.f1;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
new file mode 100644
index 0000000..60c4c17
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
@@ -0,0 +1,112 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+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.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+
+/**
+ * This is an implementation of the Single-Source-Shortest Paths algorithm, using a vertex-centric iteration.
+ */
+@SuppressWarnings("serial")
+public class SingleSourceShortestPaths<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
+
+	private final K srcVertexId;
+	private final Integer maxIterations;
+
+	public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) {
+		this.srcVertexId = srcVertexId;
+		this.maxIterations = maxIterations;
+	}
+
+	@Override
+	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> input) {
+
+		return input.mapVertices(new InitVerticesMapper<K>(srcVertexId))
+				.runVertexCentricIteration(new VertexDistanceUpdater<K>(), new MinDistanceMessenger<K>(),
+				maxIterations).getVertices();
+	}
+
+	public static final class InitVerticesMapper<K>	implements MapFunction<Vertex<K, Double>, Double> {
+
+		private K srcVertexId;
+
+		public InitVerticesMapper(K srcId) {
+			this.srcVertexId = srcId;
+		}
+
+		public Double map(Vertex<K, Double> value) {
+			if (value.f0.equals(srcVertexId)) {
+				return 0.0;
+			} else {
+				return Double.MAX_VALUE;
+			}
+		}
+	}
+
+	/**
+	 * Function that updates the value of a vertex by picking the minimum
+	 * distance from all incoming messages.
+	 * 
+	 * @param <K>
+	 */
+	public static final class VertexDistanceUpdater<K> extends VertexUpdateFunction<K, Double, Double> {
+
+		@Override
+		public void updateVertex(Vertex<K, Double> vertex,
+				MessageIterator<Double> inMessages) {
+
+			Double minDistance = Double.MAX_VALUE;
+
+			for (double msg : inMessages) {
+				if (msg < minDistance) {
+					minDistance = msg;
+				}
+			}
+
+			if (vertex.getValue() > minDistance) {
+				setNewVertexValue(minDistance);
+			}
+		}
+	}
+
+	/**
+	 * Distributes the minimum distance associated with a given vertex among all
+	 * the target vertices summed up with the edge's value.
+	 * 
+	 * @param <K>
+	 */
+	public static final class MinDistanceMessenger<K> extends MessagingFunction<K, Double, Double, Double> {
+
+		@Override
+		public void sendMessages(Vertex<K, Double> vertex)
+				throws Exception {
+			for (Edge<K, Double> edge : getEdges()) {
+				sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue());
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
new file mode 100644
index 0000000..d6fdc8a
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
@@ -0,0 +1,58 @@
+/*
+ * 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.spargel;
+
+import java.util.Iterator;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/**
+ * An iterator that returns messages. The iterator is {@link java.lang.Iterable} at the same time to support
+ * the <i>foreach</i> syntax.
+ */
+public final class MessageIterator<Message> implements Iterator<Message>, Iterable<Message>, java.io.Serializable {
+	private static final long serialVersionUID = 1L;
+
+	private transient Iterator<Tuple2<?, Message>> source;
+	
+	
+	final void setSource(Iterator<Tuple2<?, Message>> source) {
+		this.source = source;
+	}
+	
+	@Override
+	public final boolean hasNext() {
+		return this.source.hasNext();
+	}
+	
+	@Override
+	public final Message next() {
+		return this.source.next().f1;
+	}
+
+	@Override
+	public final void remove() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Iterator<Message> iterator() {
+		return this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
new file mode 100644
index 0000000..4245c24
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
@@ -0,0 +1,303 @@
+/*
+ * 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.spargel;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
+
+/**
+ * The base class for functions that produce messages between vertices as a part of a {@link VertexCentricIteration}.
+ * 
+ * @param <K> The type of the vertex key (the vertex identifier).
+ * @param <VV> The type of the vertex value (the state of the vertex).
+ * @param <Message> The type of the message sent between vertices along the edges.
+ * @param <EV> The type of the values that are associated with the edges.
+ */
+public abstract class MessagingFunction<K, VV, Message, EV> implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// --------------------------------------------------------------------------------------------
+	//  Attributes that allow vertices to access their in/out degrees and the total number of vertices
+	//  inside an iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private long numberOfVertices = -1L;
+
+	/**
+	 * Retrieves the number of vertices in the graph.
+	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
+	 * option has been set; -1 otherwise.
+	 */
+	public long getNumberOfVertices() {
+		return numberOfVertices;
+	}
+
+	void setNumberOfVertices(long numberOfVertices) {
+		this.numberOfVertices = numberOfVertices;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Attribute that allows the user to choose the neighborhood type(in/out/all) on which to run
+	//  the vertex centric iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private EdgeDirection direction;
+
+	/**
+	 * Retrieves the edge direction in which messages are propagated in the vertex-centric iteration.
+	 * @return the messaging {@link EdgeDirection}
+	 */
+	public EdgeDirection getDirection() {
+		return direction;
+	}
+
+	void setDirection(EdgeDirection direction) {
+		this.direction = direction;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Public API Methods
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * This method is invoked once per superstep for each vertex that was changed in that superstep.
+	 * It needs to produce the messages that will be received by vertices in the next superstep.
+	 * 
+	 * @param vertex The vertex that was changed.
+	 * 
+	 * @throws Exception The computation may throw exceptions, which causes the superstep to fail.
+	 */
+	public abstract void sendMessages(Vertex<K, VV> vertex) throws Exception;
+	
+	/**
+	 * This method is executed one per superstep before the vertex update function is invoked for each vertex.
+	 * 
+	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
+	 */
+	public void preSuperstep() throws Exception {}
+	
+	/**
+	 * This method is executed one per superstep after the vertex update function has been invoked for each vertex.
+	 * 
+	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
+	 */
+	public void postSuperstep() throws Exception {}
+	
+	
+	/**
+	 * Gets an {@link java.lang.Iterable} with all edges. This method is mutually exclusive with
+	 * {@link #sendMessageToAllNeighbors(Object)} and may be called only once.
+	 * 
+	 * @return An iterator with all outgoing edges.
+	 */
+	@SuppressWarnings("unchecked")
+	public Iterable<Edge<K, EV>> getEdges() {
+		if (edgesUsed) {
+			throw new IllegalStateException("Can use either 'getEdges()' or 'sendMessageToAllTargets()' exactly once.");
+		}
+		edgesUsed = true;
+		this.edgeIterator.set((Iterator<Edge<K, EV>>) edges);
+		return this.edgeIterator;
+	}
+
+	/**
+	 * Sends the given message to all vertices that are targets of an outgoing edge of the changed vertex.
+	 * This method is mutually exclusive to the method {@link #getEdges()} and may be called only once.
+	 * 
+	 * @param m The message to send.
+	 */
+	public void sendMessageToAllNeighbors(Message m) {
+		if (edgesUsed) {
+			throw new IllegalStateException("Can use either 'getEdges()' or 'sendMessageToAllTargets()' exactly once.");
+		}
+		
+		edgesUsed = true;
+		
+		outValue.f1 = m;
+		
+		while (edges.hasNext()) {
+			Tuple next = (Tuple) edges.next();
+			K k = next.getField(1);
+			outValue.f0 = k;
+			out.collect(outValue);
+		}
+	}
+	
+	/**
+	 * Sends the given message to the vertex identified by the given key. If the target vertex does not exist,
+	 * the next superstep will cause an exception due to a non-deliverable message.
+	 * 
+	 * @param target The key (id) of the target vertex to message.
+	 * @param m The message.
+	 */
+	public void sendMessageTo(K target, Message m) {
+		outValue.f0 = target;
+		outValue.f1 = m;
+		out.collect(outValue);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Gets the number of the superstep, starting at <tt>1</tt>.
+	 * 
+	 * @return The number of the current superstep.
+	 */
+	public int getSuperstepNumber() {
+		return this.runtimeContext.getSuperstepNumber();
+	}
+	
+	/**
+	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
+	 * all aggregates globally once per superstep and makes them available in the next superstep.
+	 * 
+	 * @param name The name of the aggregator.
+	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
+	 */
+	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+		return this.runtimeContext.<T>getIterationAggregator(name);
+	}
+	
+	/**
+	 * Get the aggregated value that an aggregator computed in the previous iteration.
+	 * 
+	 * @param name The name of the aggregator.
+	 * @return The aggregated value of the previous iteration.
+	 */
+	public <T extends Value> T getPreviousIterationAggregate(String name) {
+		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
+	}
+	
+	/**
+	 * Gets the broadcast data set registered under the given name. Broadcast data sets
+	 * are available on all parallel instances of a function. They can be registered via
+	 * {@link org.apache.flink.graph.spargel.VertexCentricConfiguration#addBroadcastSetForMessagingFunction(String, org.apache.flink.api.java.DataSet)}.
+	 * 
+	 * @param name The name under which the broadcast set is registered.
+	 * @return The broadcast data set.
+	 */
+	public <T> Collection<T> getBroadcastSet(String name) {
+		return this.runtimeContext.<T>getBroadcastVariable(name);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  internal methods and state
+	// --------------------------------------------------------------------------------------------
+	
+	private Tuple2<K, Message> outValue;
+	
+	private IterationRuntimeContext runtimeContext;
+	
+	private Iterator<?> edges;
+	
+	private Collector<Tuple2<K, Message>> out;
+	
+	private EdgesIterator<K, EV> edgeIterator;
+	
+	private boolean edgesUsed;
+
+	private long inDegree = -1;
+
+	private long outDegree = -1;
+	
+	void init(IterationRuntimeContext context) {
+		this.runtimeContext = context;
+		this.outValue = new Tuple2<K, Message>();
+		this.edgeIterator = new EdgesIterator<K, EV>();
+	}
+	
+	void set(Iterator<?> edges, Collector<Tuple2<K, Message>> out) {
+		this.edges = edges;
+		this.out = out;
+		this.edgesUsed = false;
+	}
+	
+	private static final class EdgesIterator<K, EV> 
+		implements Iterator<Edge<K, EV>>, Iterable<Edge<K, EV>>
+	{
+		private Iterator<Edge<K, EV>> input;
+		
+		private Edge<K, EV> edge = new Edge<K, EV>();
+		
+		void set(Iterator<Edge<K, EV>> input) {
+			this.input = input;
+		}
+		
+		@Override
+		public boolean hasNext() {
+			return input.hasNext();
+		}
+
+		@Override
+		public Edge<K, EV> next() {
+			Edge<K, EV> next = input.next();
+			edge.setSource(next.f0);
+			edge.setTarget(next.f1);
+			edge.setValue(next.f2);
+			return edge;
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException();
+		}
+		@Override
+		public Iterator<Edge<K, EV>> iterator() {
+			return this;
+		}
+	}
+
+	/**
+	 * Retrieves the vertex in-degree (number of in-coming edges).
+	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
+	 * option has been set; -1 otherwise. 
+	 */
+	public long getInDegree() {
+		return inDegree;
+	}
+
+	void setInDegree(long inDegree) {
+		this.inDegree = inDegree;
+	}
+
+	/**
+	 * Retrieve the vertex out-degree (number of out-going edges).
+	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
+	 * option has been set; -1 otherwise. 
+	 */
+	public long getOutDegree() {
+		return outDegree;
+	}
+
+	void setOutDegree(long outDegree) {
+		this.outDegree = outDegree;
+	}
+}


[04/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
deleted file mode 100644
index 5aa9f26..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
+++ /dev/null
@@ -1,101 +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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-import org.apache.flink.graph.example.MusicProfiles;
-import org.apache.flink.graph.example.utils.MusicProfilesData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.After;
-import org.junit.Assert;
-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;
-import java.util.ArrayList;
-import java.util.Arrays;
-
-@RunWith(Parameterized.class)
-public class MusicProfilesITCase extends MultipleProgramsTestBase {
-
-	private String tripletsPath;
-
-	private String mismatchesPath;
-
-	private String topSongsResultPath;
-
-	private String communitiesResultPath;
-
-	private String expectedTopSongs;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public MusicProfilesITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		topSongsResultPath = tempFolder.newFile().toURI().toString();
-		communitiesResultPath = tempFolder.newFile().toURI().toString();
-
-		File tripletsFile = tempFolder.newFile();
-		Files.write(MusicProfilesData.USER_SONG_TRIPLETS, tripletsFile, Charsets.UTF_8);
-		tripletsPath = tripletsFile.toURI().toString();
-
-		File mismatchesFile = tempFolder.newFile();
-		Files.write(MusicProfilesData.MISMATCHES, mismatchesFile, Charsets.UTF_8);
-		mismatchesPath = mismatchesFile.toURI().toString();
-	}
-
-	@Test
-	public void testMusicProfilesExample() throws Exception {
-		MusicProfiles.main(new String[]{tripletsPath, mismatchesPath, topSongsResultPath, "0", communitiesResultPath,
-				MusicProfilesData.MAX_ITERATIONS + ""});
-		expectedTopSongs = MusicProfilesData.TOP_SONGS_RESULT;
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expectedTopSongs, topSongsResultPath);
-
-		ArrayList<String> list = new ArrayList<String>();
-		readAllResultLines(list, communitiesResultPath, new String[]{}, false);
-
-		String[] result = list.toArray(new String[list.size()]);
-		Arrays.sort(result);
-
-		// check that user_1 and user_2 are in the same community
-		Assert.assertEquals("users 1 and 2 are not in the same community",
-				result[0].substring(7), result[1].substring(7));
-
-		// check that user_3, user_4 and user_5 are in the same community
-		Assert.assertEquals("users 3 and 4 are not in the same community",
-				result[2].substring(7), result[3].substring(7));
-		Assert.assertEquals("users 4 and 5 are not in the same community",
-				result[3].substring(7), result[4].substring(7));
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
deleted file mode 100644
index d8f8c8f..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
+++ /dev/null
@@ -1,81 +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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-import org.apache.flink.graph.example.GSASingleSourceShortestPaths;
-import org.apache.flink.graph.example.SingleSourceShortestPaths;
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-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 SingleSourceShortestPathsITCase extends MultipleProgramsTestBase {
-
-    private String edgesPath;
-
-    private String resultPath;
-
-    private String expected;
-
-    @Rule
-    public TemporaryFolder tempFolder = new TemporaryFolder();
-
-    public SingleSourceShortestPathsITCase(TestExecutionMode mode) {
-        super(mode);
-    }
-
-    @Before
-    public void before() throws Exception {
-        resultPath = tempFolder.newFile().toURI().toString();
-
-        File edgesFile = tempFolder.newFile();
-        Files.write(SingleSourceShortestPathsData.EDGES, edgesFile, Charsets.UTF_8);
-        edgesPath = edgesFile.toURI().toString();
-    }
-
-    @Test
-    public void testSSSPExample() throws Exception {
-        SingleSourceShortestPaths.main(new String[]{SingleSourceShortestPathsData.SRC_VERTEX_ID + "",
-                edgesPath, resultPath, 10 + ""});
-        expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS;
-    }
-
-    @Test
-    public void testGSASSSPExample() throws Exception {
-        GSASingleSourceShortestPaths.main(new String[]{SingleSourceShortestPathsData.SRC_VERTEX_ID + "",
-                edgesPath, resultPath, 10 + ""});
-        expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS;
-    }
-
-    @After
-    public void after() throws Exception {
-        compareResultsByLinesInMemory(expected, resultPath);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
deleted file mode 100644
index 421eaa9..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
+++ /dev/null
@@ -1,82 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.CommunityDetectionData;
-import org.apache.flink.graph.library.CommunityDetection;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class CommunityDetectionITCase extends MultipleProgramsTestBase {
-
-	public CommunityDetectionITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private String expected;
-
-	@Test
-	public void testSingleIteration() throws Exception {
-		/*
-		 * Test one iteration of the Simple Community Detection Example
-		 */
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
-				CommunityDetectionData.getSimpleEdgeDataSet(env), new InitLabels(), env);
-
-        List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
-        		.getVertices().collect();
-
-		expected = CommunityDetectionData.COMMUNITIES_SINGLE_ITERATION;
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testTieBreaker() throws Exception {
-		/*
-		 * Test one iteration of the Simple Community Detection Example where a tie must be broken
-		 */
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
-				CommunityDetectionData.getTieEdgeDataSet(env), new InitLabels(), env);
-
-        List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
-        		.getVertices().collect();
-		expected = CommunityDetectionData.COMMUNITIES_WITH_TIE;
-		compareResultAsTuples(result, expected);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitLabels implements MapFunction<Long, Long>{
-
-		public Long map(Long id) {
-			return id;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
deleted file mode 100644
index 9eb7387..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
+++ /dev/null
@@ -1,94 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.library.ConnectedComponents;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.types.NullValue;
-
-import java.io.BufferedReader;
-
-@SuppressWarnings("serial")
-public class ConnectedComponentsWithRandomisedEdgesITCase extends JavaProgramTestBase {
-
-	private static final long SEED = 9487520347802987L;
-
-	private static final int NUM_VERTICES = 1000;
-
-	private static final int NUM_EDGES = 10000;
-
-	private String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Long> vertexIds = env.generateSequence(1, NUM_VERTICES);
-		DataSet<String> edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED).split("\n"));
-
-		DataSet<Edge<Long, NullValue>> edges = edgeString.map(new EdgeParser());
-
-		DataSet<Vertex<Long, Long>> initialVertices = vertexIds.map(new IdAssigner());
-
-		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
-
-		DataSet<Vertex<Long, Long>> result = graph.run(new ConnectedComponents<Long, NullValue>(100));
-
-		result.writeAsCsv(resultPath, "\n", " ");
-		env.execute();
-	}
-
-	/**
-	 * A map function that takes a Long value and creates a 2-tuple out of it:
-	 * <pre>(Long value) -> (value, value)</pre>
-	 */
-	public static final class IdAssigner implements MapFunction<Long, Vertex<Long, Long>> {
-		@Override
-		public Vertex<Long, Long> map(Long value) {
-			return new Vertex<Long, Long>(value, value);
-		}
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		for (BufferedReader reader : getResultReader(resultPath)) {
-			ConnectedComponentsData.checkOddEvenResult(reader);
-		}
-	}
-
-	public static final class EdgeParser extends RichMapFunction<String, Edge<Long, NullValue>> {
-		public Edge<Long, NullValue> map(String value) {
-			String[] nums = value.split(" ");
-			return new Edge<Long, NullValue>(Long.parseLong(nums[0]), Long.parseLong(nums[1]),
-					NullValue.getInstance());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
deleted file mode 100644
index 8785b0d..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
+++ /dev/null
@@ -1,78 +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.library;
-
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.LabelPropagationData;
-import org.apache.flink.graph.library.LabelPropagation;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class LabelPropagationITCase extends MultipleProgramsTestBase {
-
-	public LabelPropagationITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testSingleIteration() throws Exception {
-		/*
-		 * Test one iteration of label propagation example with a simple graph
-		 */
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
-				LabelPropagationData.getDefaultVertexSet(env),
-				LabelPropagationData.getDefaultEdgeDataSet(env), env);
-
-        List<Vertex<Long, Long>> result = inputGraph.run(new LabelPropagation<Long, NullValue>(1))
-        		.collect();
-
-		expectedResult = LabelPropagationData.LABELS_AFTER_1_ITERATION;
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testTieBreaker() throws Exception {
-		/*
-		 * Test the label propagation example where a tie must be broken
-		 */
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
-				LabelPropagationData.getTieVertexSet(env),
-				LabelPropagationData.getTieEdgeDataSet(env), env);
-
-        List<Vertex<Long, Long>> result = inputGraph.run(new LabelPropagation<Long, NullValue>(1))
-        		.collect();
-
-		expectedResult = LabelPropagationData.LABELS_WITH_TIE;
-		compareResultAsTuples(result, expectedResult);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
deleted file mode 100644
index 94c7713..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
+++ /dev/null
@@ -1,132 +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.library;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.PageRankData;
-import org.apache.flink.graph.library.GSAPageRank;
-import org.apache.flink.graph.library.PageRank;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class PageRankITCase extends MultipleProgramsTestBase {
-
-	public PageRankITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	private String expectedResult;
-
-	@Test
-	public void testPageRankWithThreeIterations() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
-				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
-
-        List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 3))
-        		.collect();
-        
-        compareWithDelta(result, expectedResult, 0.01);
-	}
-
-	@Test
-	public void testGSAPageRankWithThreeIterations() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
-				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
-
-        List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 3))
-        		.collect();
-        
-        compareWithDelta(result, expectedResult, 0.01);
-	}
-
-	@Test
-	public void testPageRankWithThreeIterationsAndNumOfVertices() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
-				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
-
-        List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 5, 3))
-        		.collect();
-        
-        compareWithDelta(result, expectedResult, 0.01);
-	}
-
-	@Test
-	public void testGSAPageRankWithThreeIterationsAndNumOfVertices() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
-				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
-
-        List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 5, 3))
-        		.collect();
-        
-        compareWithDelta(result, expectedResult, 0.01);
-	}
-
-	private void compareWithDelta(List<Vertex<Long, Double>> result,
-			String expectedResult, double delta) {
-
-		String resultString = "";
-        for (Vertex<Long, Double> v : result) {
-        	resultString += v.f0.toString() + "," + v.f1.toString() +"\n";
-        }
-        
-		expectedResult = PageRankData.RANKS_AFTER_3_ITERATIONS;
-		String[] expected = expectedResult.isEmpty() ? new String[0] : expectedResult.split("\n");
-
-		String[] resultArray = resultString.isEmpty() ? new String[0] : resultString.split("\n");
-
-		Arrays.sort(expected);
-        Arrays.sort(resultArray);
-
-		for (int i = 0; i < expected.length; i++) {
-			String[] expectedFields = expected[i].split(",");
-			String[] resultFields = resultArray[i].split(",");
-
-			double expectedPayLoad = Double.parseDouble(expectedFields[1]);
-			double resultPayLoad = Double.parseDouble(resultFields[1]);
-
-			Assert.assertTrue("Values differ by more than the permissible delta",
-					Math.abs(expectedPayLoad - resultPayLoad) < delta);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitMapper implements MapFunction<Long, Double> {
-		public Double map(Long value) {
-			return 1.0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java
deleted file mode 100644
index 1d9ab9f..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java
+++ /dev/null
@@ -1,56 +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.library;
-
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.example.utils.TriangleCountData;
-import org.apache.flink.graph.library.GSATriangleCount;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class TriangleCountITCase extends MultipleProgramsTestBase {
-
-	private String expectedResult;
-
-	public TriangleCountITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Test
-	public void testGSATriangleCount() throws Exception {
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(TriangleCountData.getDefaultEdgeDataSet(env),
-				env).getUndirected();
-
-		List<Integer> numberOfTriangles = graph.run(new GSATriangleCount<Long, NullValue, NullValue>()).collect();
-		expectedResult = TriangleCountData.RESULTED_NUMBER_OF_TRIANGLES;
-
-		Assert.assertEquals(numberOfTriangles.get(0).intValue(), Integer.parseInt(expectedResult));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
deleted file mode 100644
index b2744f9..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
+++ /dev/null
@@ -1,178 +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.operations;
-
-import java.util.List;
-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.Graph;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class DegreesITCase extends MultipleProgramsTestBase {
-
-	public DegreesITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-
-	@Test
-	public void testOutDegrees() throws Exception {
-		/*
-		* Test outDegrees()
-		*/
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-       
-        
-        expectedResult = "1,2\n" +
-                    "2,1\n" +
-                    "3,2\n" +
-                    "4,1\n" +
-                    "5,1\n";
-        
-        compareResultAsTuples(result, expectedResult);
-        
-    }
-
-	@Test
-	public void testOutDegreesWithNoOutEdges() throws Exception {
-		/*
-		 * Test outDegrees() no outgoing edges
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env);
-
-        
-        
-        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-        
-        expectedResult = "1,3\n" +
-                "2,1\n" +
-                "3,1\n" +
-                "4,1\n" +
-                "5,0\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testInDegrees() throws Exception {
-		/*
-		 * Test inDegrees()
-		 */
-	    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-	    Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-	            TestGraphUtils.getLongLongEdgeData(env), env);
-
-
-        DataSet<Tuple2<Long,Long>> data =graph.inDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-	    
-	    expectedResult = "1,1\n" +
-		            "2,1\n" +
-		            "3,2\n" +
-		            "4,1\n" +
-		            "5,2\n";
-	    compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testInDegreesWithNoInEdge() throws Exception {
-		/*
-		 * Test inDegrees() no ingoing edge
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env);
-
-        DataSet<Tuple2<Long,Long>> data =graph.inDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-        
-        expectedResult = "1,0\n" +
-	                "2,1\n" +
-	                "3,1\n" +
-	                "4,1\n" +
-	                "5,3\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testGetDegrees() throws Exception {
-		/*
-		 * Test getDegrees()
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Tuple2<Long,Long>> data =graph.getDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-        
-        expectedResult = "1,3\n" +
-	                "2,2\n" +
-	                "3,4\n" +
-	                "4,2\n" +
-	                "5,3\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testGetDegreesWithDisconnectedData() throws Exception {
-        /*
-		 * Test getDegrees() with disconnected data
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, NullValue, Long> graph =
-                Graph.fromDataSet(TestGraphUtils.getDisconnectedLongLongEdgeData(env), env);
-
-        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
-        List<Tuple2<Long,Long>> result= data.collect();
-        
-        expectedResult = "1,2\n" +
-                "2,1\n" +
-                "3,0\n" +
-                "4,1\n" +
-                "5,0\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
deleted file mode 100644
index 955122f..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ /dev/null
@@ -1,189 +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.operations;
-
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class DegreesWithExceptionITCase {
-
-	private static final int PARALLELISM = 4;
-
-	private static ForkableFlinkMiniCluster cluster;
-	
-
-	@BeforeClass
-	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
-		}
-	}
-
-	/**
-	 * Test outDegrees() with an edge having a srcId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testOutDegreesInvalidEdgeSrcId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
-
-		try {
-			graph.outDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-
-			fail("graph.outDegrees() did not fail.");
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test inDegrees() with an edge having a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testInDegreesInvalidEdgeTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
-
-		try {
-			graph.inDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-
-			fail("graph.inDegrees() did not fail.");
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test getDegrees() with an edge having a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGetDegreesInvalidEdgeTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
-
-		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-
-			fail("graph.getDegrees() did not fail.");
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test getDegrees() with an edge having a srcId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGetDegreesInvalidEdgeSrcId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
-
-		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-
-			fail("graph.getDegrees() did not fail.");
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test getDegrees() with an edge having a srcId and a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGetDegreesInvalidEdgeSrcTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcTrgData(env), env);
-
-		try {
-			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-
-			fail("graph.getDegrees() did not fail.");
-		}
-		catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
deleted file mode 100644
index 5a64dd7..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
+++ /dev/null
@@ -1,118 +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.operations;
-
-import java.util.List;
-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.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class FromCollectionITCase extends MultipleProgramsTestBase {
-
-	public FromCollectionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-
-	@Test
-	public void testFromCollectionVerticesEdges() throws Exception {
-		/*
-		 * Test fromCollection(vertices, edges):
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-                TestGraphUtils.getLongLongEdges(), env);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-        
-        expectedResult = "1,2,12\n" +
-	                "1,3,13\n" +
-	                "2,3,23\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testFromCollectionEdgesNoInitialValue() throws Exception {
-        /*
-         * Test fromCollection(edges) with no initial value for the vertices
-         */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-        Graph<Long, NullValue, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(),
-        		env);
-
-        
-        DataSet<Vertex<Long,NullValue>> data = graph.getVertices();
-        List<Vertex<Long,NullValue>> result= data.collect();
-        
-        expectedResult = "1,(null)\n" +
-	                "2,(null)\n" +
-	                "3,(null)\n" +
-	                "4,(null)\n" +
-	                "5,(null)\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testFromCollectionEdgesWithInitialValue() throws Exception {
-        /*
-         * Test fromCollection(edges) with vertices initialised by a
-         * function that takes the id and doubles it
-         */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(),
-                new InitVerticesMapper(), env);
-
-        DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-        
-        expectedResult = "1,2\n" +
-	                "2,4\n" +
-	                "3,6\n" +
-	                "4,8\n" +
-	                "5,10\n";
-        
-        compareResultAsTuples(result, expectedResult);
-    }
-
-	@SuppressWarnings("serial")
-	private static final class InitVerticesMapper implements MapFunction<Long, Long> {
-        public Long map(Long vertexId) {
-            return vertexId * 2;
-        }
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
deleted file mode 100644
index 22a5151..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
+++ /dev/null
@@ -1,174 +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.operations;
-
-import java.util.LinkedList;
-import java.util.List;
-
-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.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
-import org.apache.flink.graph.validation.InvalidVertexIdsValidator;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class GraphCreationITCase extends MultipleProgramsTestBase {
-
-	public GraphCreationITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-
-    private String expectedResult;
-
-	@Test
-	public void testCreateWithoutVertexValues() throws Exception {
-	/*
-	 * Test create() with edge dataset and no vertex values
-     */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, NullValue, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Vertex<Long,NullValue>> data = graph.getVertices();
-        List<Vertex<Long,NullValue>> result= data.collect();
-        
-		expectedResult = "1,(null)\n" +
-					"2,(null)\n" +
-					"3,(null)\n" +
-					"4,(null)\n" +
-					"5,(null)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCreateWithMapper() throws Exception {
-	/*
-	 * Test create() with edge dataset and a mapper that assigns the id as value
-     */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env),
-				new AssignIdAsValueMapper(), env);
-
-        DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-        
-		expectedResult = "1,1\n" +
-					"2,2\n" +
-					"3,3\n" +
-					"4,4\n" +
-					"5,5\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCreateWithCustomVertexValue() throws Exception {
-		/*
-		 * Test create() with edge dataset and a mapper that assigns a parametrized custom vertex value
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, DummyCustomParameterizedType<Double>, Long> graph = Graph.fromDataSet(
-				TestGraphUtils.getLongLongEdgeData(env), new AssignCustomVertexValueMapper(), env);
-
-        DataSet<Vertex<Long,DummyCustomParameterizedType<Double>>> data = graph.getVertices();
-        List<Vertex<Long,DummyCustomParameterizedType<Double>>> result= data.collect();
-        
-		expectedResult = "1,(2.0,0)\n" +
-				"2,(4.0,1)\n" +
-				"3,(6.0,2)\n" +
-				"4,(8.0,3)\n" +
-				"5,(10.0,4)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testValidate() throws Exception {
-		/*
-		 * Test validate():
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Vertex<Long, Long>> vertices = TestGraphUtils.getLongLongVertexData(env);
-		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
-		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
-
-		//env.fromElements(result).writeAsText(resultPath);
-		
-		String res= valid.toString();//env.fromElements(valid);
-        List<String> result= new LinkedList<String>();
-        result.add(res);
-		expectedResult = "true";
-		
-		compareResultAsText(result, expectedResult);
-	}
-
-	@Test
-	public void testValidateWithInvalidIds() throws Exception {
-		/*
-		 * Test validate() - invalid vertex ids
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Vertex<Long, Long>> vertices = TestGraphUtils.getLongLongInvalidVertexData(env);
-		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
-		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
-		
-		String res= valid.toString();//env.fromElements(valid);
-        List<String> result= new LinkedList<String>();
-        result.add(res);
-
-		expectedResult = "false\n";
-		
-		compareResultAsText(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignIdAsValueMapper implements MapFunction<Long, Long> {
-		public Long map(Long vertexId) {
-			return vertexId;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignCustomVertexValueMapper implements
-		MapFunction<Long, DummyCustomParameterizedType<Double>> {
-
-		DummyCustomParameterizedType<Double> dummyValue =
-				new DummyCustomParameterizedType<Double>();
-
-		public DummyCustomParameterizedType<Double> map(Long vertexId) {
-			dummyValue.setIntField(vertexId.intValue()-1);
-			dummyValue.setTField(vertexId*2.0);
-			return dummyValue;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
deleted file mode 100644
index 99c66ec..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
+++ /dev/null
@@ -1,204 +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.operations;
-
-import com.google.common.base.Charsets;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Triplet;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class GraphCreationWithCsvITCase extends MultipleProgramsTestBase {
-
-	public GraphCreationWithCsvITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private String expectedResult;
-
-	@Test
-	public void testCreateWithCsvFile() throws Exception {
-		/*
-		 * Test with two Csv files one with Vertex Data and one with Edges data
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final String fileContent =  "1,1\n"+
-				"2,2\n"+
-				"3,3\n";
-		final FileInputSplit split = createTempFile(fileContent);
-		final String fileContent2 =  "1,2,ot\n"+
-				"3,2,tt\n"+
-				"3,1,to\n";
-		final FileInputSplit split2 = createTempFile(fileContent2);
-
-		Graph<Long, Long, String> graph = Graph.fromCsvReader(split.getPath().toString(),split2.getPath().toString(),env)
-				.types(Long.class, Long.class, String.class);
-
-		List<Triplet<Long, Long, String>> result = graph.getTriplets().collect();
-
-		expectedResult = "1,2,1,2,ot\n" +
-				"3,2,3,2,tt\n" +
-				"3,1,3,1,to\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCsvWithNullEdge() throws Exception {
-		/*
-		Test fromCsvReader with edge and vertex path and nullvalue for edge
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final String vertexFileContent = "1,one\n"+
-				"2,two\n"+
-				"3,three\n";
-		final String edgeFileContent = "1,2\n"+
-				"3,2\n"+
-				"3,1\n";
-		final FileInputSplit split = createTempFile(vertexFileContent);
-		final FileInputSplit edgeSplit = createTempFile(edgeFileContent);
-
-		Graph<Long, String, NullValue> graph = Graph.fromCsvReader(split.getPath().toString(), edgeSplit.getPath().toString(),
-				env).vertexTypes(Long.class, String.class);
-
-		List<Triplet<Long, String, NullValue>> result = graph.getTriplets().collect();
-
-		expectedResult = "1,2,one,two,(null)\n"+
-				"3,2,three,two,(null)\n"+
-				"3,1,three,one,(null)\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCsvWithConstantValueMapper() throws Exception {
-		/*
-		*Test fromCsvReader with edge path and a mapper that assigns a Double constant as value
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final String fileContent =  "1,2,ot\n"+
-				"3,2,tt\n"+
-				"3,1,to\n";
-		final FileInputSplit split = createTempFile(fileContent);
-
-		Graph<Long, Double, String> graph = Graph.fromCsvReader(split.getPath().toString(),
-				new AssignDoubleValueMapper(), env).types(Long.class, Double.class, String.class);
-
-		List<Triplet<Long, Double, String>> result = graph.getTriplets().collect();
-		//graph.getTriplets().writeAsCsv(resultPath);
-		expectedResult = "1,2,0.1,0.1,ot\n" + "3,1,0.1,0.1,to\n" + "3,2,0.1,0.1,tt\n";
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCreateWithOnlyEdgesCsvFile() throws Exception {
-		/*
-		 * Test with one Csv file one with Edges data. Also tests the configuration method ignoreFistLineEdges()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final String fileContent2 =  "header\n1,2,ot\n"+
-				"3,2,tt\n"+
-				"3,1,to\n";
-
-		final FileInputSplit split2 = createTempFile(fileContent2);
-		Graph<Long, NullValue, String> graph= Graph.fromCsvReader(split2.getPath().toString(), env)
-				.ignoreFirstLineEdges()
-				.ignoreCommentsVertices("hi")
-				.edgeTypes(Long.class, String.class);
-
-		List<Triplet<Long, NullValue, String>> result = graph.getTriplets().collect();
-		expectedResult = "1,2,(null),(null),ot\n" +
-				"3,2,(null),(null),tt\n" +
-				"3,1,(null),(null),to\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testCreateCsvFileDelimiterConfiguration() throws Exception {
-		/*
-		 * Test with an Edge and Vertex csv file. Tests the configuration methods FieldDelimiterEdges and
-		 * FieldDelimiterVertices
-		 * Also tests the configuration methods LineDelimiterEdges and LineDelimiterVertices
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		final String fileContent =  "header\n1;1\n"+
-				"2;2\n"+
-				"3;3\n";
-
-		final FileInputSplit split = createTempFile(fileContent);
-
-		final String fileContent2 =  "header|1:2:ot|"+
-				"3:2:tt|"+
-				"3:1:to|";
-
-		final FileInputSplit split2 = createTempFile(fileContent2);
-
-		Graph<Long, Long, String> graph= Graph.fromCsvReader(split.getPath().toString(),split2.getPath().toString(),env).
-				ignoreFirstLineEdges().ignoreFirstLineVertices().
-				fieldDelimiterEdges(":").fieldDelimiterVertices(";").
-				lineDelimiterEdges("|").
-				types(Long.class, Long.class, String.class);
-
-		List<Triplet<Long, Long, String>> result = graph.getTriplets().collect();
-
-		expectedResult = "1,2,1,2,ot\n" +
-				"3,2,3,2,tt\n" +
-				"3,1,3,1,to\n";
-
-		compareResultAsTuples(result, expectedResult);
-
-	}
-
-	/*----------------------------------------------------------------------------------------------------------------*/
-	@SuppressWarnings("serial")
-	private static final class AssignDoubleValueMapper implements MapFunction<Long, Double> {
-		public Double map(Long value) {
-			return 0.1d;
-		}
-	}
-
-	private FileInputSplit createTempFile(String content) throws IOException {
-		File tempFile = File.createTempFile("test_contents", "tmp");
-		tempFile.deleteOnExit();
-
-		OutputStreamWriter wrt = new OutputStreamWriter(
-				new FileOutputStream(tempFile), Charsets.UTF_8
-		);
-		wrt.write(content);
-		wrt.close();
-
-		return new FileInputSplit(0, new Path(tempFile.toURI().toString()), 0,
-							tempFile.length(), new String[] {"localhost"});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
deleted file mode 100644
index 20cbca5..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
+++ /dev/null
@@ -1,158 +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.operations;
-
-import java.util.List;
-
-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.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class GraphCreationWithMapperITCase extends MultipleProgramsTestBase {
-
-	public GraphCreationWithMapperITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-
-	@Test
-	public void testWithDoubleValueMapper() throws Exception {
-		/*
-		 * Test create() with edge dataset and a mapper that assigns a double constant as value
-	     */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Double, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env),
-				new AssignDoubleValueMapper(), env);
-
-        DataSet<Vertex<Long,Double>> data = graph.getVertices();
-        List<Vertex<Long,Double>> result= data.collect();
-		
-		expectedResult = "1,0.1\n" +
-				"2,0.1\n" +
-				"3,0.1\n" +
-				"4,0.1\n" +
-				"5,0.1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithTuple2ValueMapper() throws Exception {
-		/*
-		 * Test create() with edge dataset and a mapper that assigns a Tuple2 as value
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Tuple2<Long, Long>, Long> graph = Graph.fromDataSet(
-				TestGraphUtils.getLongLongEdgeData(env), new AssignTuple2ValueMapper(), env);
-
-        DataSet<Vertex<Long, Tuple2<Long, Long>>> data = graph.getVertices();
-        List<Vertex<Long, Tuple2<Long, Long>>> result= data.collect();
-        
-		expectedResult = "1,(2,42)\n" +
-				"2,(4,42)\n" +
-				"3,(6,42)\n" +
-				"4,(8,42)\n" +
-				"5,(10,42)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithConstantValueMapper() throws Exception {
-	/*
-	 * Test create() with edge dataset with String key type
-	 * and a mapper that assigns a double constant as value
-	 */
-	final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-	Graph<String, Double, Long> graph = Graph.fromDataSet(TestGraphUtils.getStringLongEdgeData(env),
-			new AssignDoubleConstantMapper(), env);
-
-    DataSet<Vertex<String,Double>> data = graph.getVertices();
-    List<Vertex<String,Double>> result= data.collect();
-    
-	expectedResult = "1,0.1\n" +
-			"2,0.1\n" +
-			"3,0.1\n" +
-			"4,0.1\n" +
-			"5,0.1\n";
-	
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithDCustomValueMapper() throws Exception {
-		/*
-		 * Test create() with edge dataset and a mapper that assigns a custom vertex value
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, DummyCustomType, Long> graph = Graph.fromDataSet(
-				TestGraphUtils.getLongLongEdgeData(env), new AssignCustomValueMapper(), env);
-
-	    DataSet<Vertex<Long,DummyCustomType>> data = graph.getVertices();
-	    List<Vertex<Long,DummyCustomType>> result= data.collect();
-	    
-		expectedResult = "1,(F,0)\n" +
-				"2,(F,1)\n" +
-				"3,(F,2)\n" +
-				"4,(F,3)\n" +
-				"5,(F,4)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignDoubleValueMapper implements MapFunction<Long, Double> {
-		public Double map(Long value) {
-			return 0.1d;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignTuple2ValueMapper implements MapFunction<Long, Tuple2<Long, Long>> {
-		public Tuple2<Long, Long> map(Long vertexId) {
-			return new Tuple2<Long, Long>(vertexId*2, 42l);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignDoubleConstantMapper implements MapFunction<String, Double> {
-		public Double map(String value) {
-			return 0.1d;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AssignCustomValueMapper implements MapFunction<Long, DummyCustomType> {
-		public DummyCustomType map(Long vertexId) {
-			return new DummyCustomType(vertexId.intValue()-1, false);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
deleted file mode 100644
index d6e5a9c..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
+++ /dev/null
@@ -1,603 +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.operations;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class GraphMutationsITCase extends MultipleProgramsTestBase {
-
-	public GraphMutationsITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-
-	@Test
-	public void testAddVertex() throws Exception {
-		/*
-		 * Test addVertex() -- simple case
-		 */	
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		graph = graph.addVertex(new Vertex<Long, Long>(6L, 6L));
-        
-		DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n" +
-				"6,6\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddVertices() throws Exception {
-		/*
-		 * Test addVertices() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(6L, 6L));
-		vertices.add(new Vertex<Long, Long>(7L, 7L));
-
-		graph = graph.addVertices(vertices);
-
-		DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n" +
-				"6,6\n" +
-				"7,7\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddVertexExisting() throws Exception {
-		/*
-		 * Test addVertex() -- add an existing vertex
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		graph = graph.addVertex(new Vertex<Long, Long>(1L, 1L));
-		
-		DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddVerticesBothExisting() throws Exception {
-		/*
-		 * Test addVertices() -- add two existing vertices
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 1L));
-		vertices.add(new Vertex<Long, Long>(3L, 3L));
-
-		graph = graph.addVertices(vertices);
-
-		DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddVerticesOneExisting() throws Exception {
-		/*
-		 * Test addVertices() -- add an existing vertex
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 1L));
-		vertices.add(new Vertex<Long, Long>(6L, 6L));
-
-		graph = graph.addVertices(vertices);
-
-		DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n" +
-				"6,6\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveVertex() throws Exception {
-		/*
-		 * Test removeVertex() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.removeVertex(new Vertex<Long, Long>(5L, 5L));
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveVertices() throws Exception {
-		/*
-		 * Test removeVertices() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
-		verticesToBeRemoved.add(new Vertex<Long, Long>(1L, 1L));
-		verticesToBeRemoved.add(new Vertex<Long, Long>(2L, 2L));
-
-		graph = graph.removeVertices(verticesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveInvalidVertex() throws Exception {
-		/*
-		 * Test removeVertex() -- remove an invalid vertex
-		 */	
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.removeVertex(new Vertex<Long, Long>(6L, 6L));
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveOneValidOneInvalidVertex() throws Exception {
-		/*
-		 * Test removeVertices() -- remove one invalid vertex and a valid one
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
-		verticesToBeRemoved.add(new Vertex<Long, Long>(1L, 1L));
-		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
-
-		graph = graph.removeVertices(verticesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveBothInvalidVertices() throws Exception {
-		/*
-		 * Test removeVertices() -- remove two invalid vertices
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
-		verticesToBeRemoved.add(new Vertex<Long, Long>(6L, 6L));
-		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
-
-		graph = graph.removeVertices(verticesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveBothInvalidVerticesVertexResult() throws Exception {
-		/*
-		 * Test removeVertices() -- remove two invalid vertices and verify the data set of vertices
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
-		verticesToBeRemoved.add(new Vertex<Long, Long>(6L, 6L));
-		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
-
-		graph = graph.removeVertices(verticesToBeRemoved);
-
-        DataSet<Vertex<Long,Long>> data = graph.getVertices();
-        List<Vertex<Long, Long>> result= data.collect();
-
-		expectedResult = "1,1\n" +
-				"2,2\n" +
-				"3,3\n" +
-				"4,4\n" +
-				"5,5\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-	
-	@Test
-	public void testAddEdge() throws Exception {
-		/*
-		 * Test addEdge() -- simple case
-		 */
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.addEdge(new Vertex<Long, Long>(6L, 6L), new Vertex<Long, Long>(1L, 1L),
-				61L);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n" +
-				"6,1,61\n";	
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddEdges() throws Exception {
-		/*
-		 * Test addEdges() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Edge<Long, Long>> edgesToBeAdded = new ArrayList<Edge<Long, Long>>();
-		edgesToBeAdded.add(new Edge<Long, Long>(2L, 4L, 24L));
-		edgesToBeAdded.add(new Edge<Long, Long>(4L, 1L, 41L));
-
-		graph = graph.addEdges(edgesToBeAdded);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"2,4,24\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,1,41\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddEdgesInvalidVertices() throws Exception {
-		/*
-		 * Test addEdges() -- the source and target vertices do not exist in the graph
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Edge<Long, Long>> edgesToBeAdded = new ArrayList<Edge<Long, Long>>();
-		edgesToBeAdded.add(new Edge<Long, Long>(6L, 1L, 61L));
-		edgesToBeAdded.add(new Edge<Long, Long>(7L, 1L, 71L));
-
-		graph = graph.addEdges(edgesToBeAdded);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAddExistingEdge() throws Exception {
-		/*
-		 * Test addEdge() -- add already existing edge
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.addEdge(new Vertex<Long, Long>(1L, 1L), new Vertex<Long, Long>(2L, 2L),
-				12L);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";	
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveEdge() throws Exception {
-		/*
-		 * Test removeEdge() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.removeEdge(new Edge<Long, Long>(5L, 1L, 51L));
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveEdges() throws Exception {
-		/*
-		 * Test removeEdges() -- simple case
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
-		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
-		edgesToBeRemoved.add(new Edge<Long, Long>(2L, 3L, 23L));
-
-		graph = graph.removeEdges(edgesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveSameEdgeTwice() throws Exception {
-		/*
-		 * Test removeEdges() -- try to remove the same edge twice
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
-		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
-		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
-
-		graph = graph.removeEdges(edgesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveInvalidEdge() throws Exception {
-		/*
-		 * Test removeEdge() -- invalid edge
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		graph = graph.removeEdge(new Edge<Long, Long>(6L, 1L, 61L));
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testRemoveOneValidOneInvalidEdge() throws Exception {
-		/*
-		 * Test removeEdges() -- one edge is valid, the other is invalid
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
-		edgesToBeRemoved.add(new Edge<Long, Long>(1L, 1L, 51L));
-		edgesToBeRemoved.add(new Edge<Long, Long>(6L, 1L, 61L));
-
-		graph = graph.removeEdges(edgesToBeRemoved);
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-}
\ No newline at end of file


[23/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
new file mode 100644
index 0000000..2dc272c
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
@@ -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.scala.example;
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.types.NullValue
+import org.apache.flink.graph.Edge
+import org.apache.flink.api.common.functions.MapFunction
+import scala.collection.JavaConversions._
+import org.apache.flink.graph.scala.utils.Tuple3ToEdgeMap
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData
+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.gsa.ApplyFunction
+
+/**
+ * This example shows how to use Gelly's gather-sum-apply iterations.
+ * 
+ * It is an implementation of the Single-Source-Shortest-Paths algorithm. 
+ *
+ * The input file is a plain text file and must be formatted as follows:
+ * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
+ * separated by tabs. Edges themselves are separated by newlines.
+ * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
+ * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.graph.example.utils.SingleSourceShortestPathsData]]
+ */
+object GSASingleSourceShortestPaths {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edges: DataSet[Edge[Long, Double]] = getEdgesDataSet(env)
+    val graph = Graph.fromDataSet[Long, Double, Double](edges, new InitVertices(srcVertexId), env)
+
+    // Execute the gather-sum-apply iteration
+    val result = graph.runGatherSumApplyIteration(new CalculateDistances, new ChooseMinDistance,
+      new UpdateDistance, maxIterations)
+
+    // Extract the vertices as the result
+    val singleSourceShortestPaths = result.getVertices
+
+    // emit result
+    if (fileOutput) {
+      singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",")
+      env.execute("GSA Single Source Shortest Paths Example")
+    } else {
+      singleSourceShortestPaths.print()
+    }
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Single Source Shortest Path UDFs
+  // --------------------------------------------------------------------------------------------
+
+  private final class InitVertices(srcId: Long) extends MapFunction[Long, Double] {
+
+    override def map(id: Long) = {
+      if (id.equals(srcId)) {
+        0.0
+      } else {
+        Double.PositiveInfinity
+      }
+    }
+  }
+
+  private final class CalculateDistances extends GatherFunction[Double, Double, Double] {
+    override def gather(neighbor: Neighbor[Double, Double]) = {
+      neighbor.getNeighborValue + neighbor.getEdgeValue
+    }
+  }
+
+  private final class ChooseMinDistance extends SumFunction[Double, Double, Double] {
+    override def sum(newValue: Double, currentValue: Double) = {
+      Math.min(newValue, currentValue)
+    }
+  }
+
+  private final class UpdateDistance extends ApplyFunction[Long, Double, Double] {
+    override def apply(newDistance: Double, oldDistance: Double) = {
+      if (newDistance < oldDistance) {
+        setResult(newDistance)
+      }
+    }
+  }
+
+  // **************************************************************************
+  // UTIL METHODS
+  // **************************************************************************
+
+  private var fileOutput = false
+  private var srcVertexId = 1L
+  private var edgesInputPath: String = null
+  private var outputPath: String = null
+  private var maxIterations = 5
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if(args.length > 0) {
+      if(args.length != 4) {
+        System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+          " <input edges path> <output path> <num iterations>")
+        false
+      }
+      fileOutput = true
+      srcVertexId = args(0).toLong
+      edgesInputPath = args(1)
+      outputPath = args(2)
+      maxIterations = (3).toInt
+    } else {
+      System.out.println("Executing Single Source Shortest Paths example "
+        + "with default parameters and built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+        " <input edges path> <output path> <num iterations>");
+    }
+    true
+  }
+
+  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, Double]] = {
+    if (fileOutput) {
+      env.readCsvFile[(Long, Long, Double)](edgesInputPath,
+        lineDelimiter = "\n",
+        fieldDelimiter = "\t")
+        .map(new Tuple3ToEdgeMap[Long, Double]())
+    } else {
+      val edgeData = SingleSourceShortestPathsData.DEFAULT_EDGES map {
+        case Array(x, y, z) => (x.asInstanceOf[Long], y.asInstanceOf[Long],
+          z.asInstanceOf[Double])
+      }
+      env.fromCollection(edgeData).map(new Tuple3ToEdgeMap[Long, Double]())
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
new file mode 100644
index 0000000..4eed824
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
@@ -0,0 +1,128 @@
+/*
+ * 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.scala.example
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.types.NullValue
+import org.apache.flink.graph.Edge
+import org.apache.flink.util.Collector
+
+/**
+ * This example illustrates how to use Gelly metrics methods and get simple statistics
+ * from the input graph.  
+ * 
+ * The program creates a random graph and computes and prints
+ * the following metrics:
+ * - number of vertices
+ * - number of edges
+ * - average node degree
+ * - the vertex ids with the max/min in- and out-degrees
+ *
+ * The input file is expected to contain one edge per line,
+ * with long IDs and no values, in the following format:
+ * {{{
+ *   <sourceVertexID>\t<targetVertexID>
+ * }}}
+ * If no arguments are provided, the example runs with a random graph of 100 vertices.
+ *
+ */
+object GraphMetrics {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    /** create the graph **/
+    val graph: Graph[Long, NullValue, NullValue] = Graph.fromDataSet(getEdgeDataSet(env), env)
+
+    /** get the number of vertices **/
+    val numVertices = graph.numberOfVertices;
+
+    /** get the number of edges **/
+    val numEdges = graph.numberOfEdges;
+
+    /** compute the average node degree **/
+    val verticesWithDegrees = graph.getDegrees;
+    val avgDegree = verticesWithDegrees.sum(1).map(in => (in._2 / numVertices).toDouble)
+
+    /** find the vertex with the maximum in-degree **/
+    val maxInDegreeVertex = graph.inDegrees.max(1).map(in => in._1)
+
+    /** find the vertex with the minimum in-degree **/
+    val minInDegreeVertex = graph.inDegrees.min(1).map(in => in._1)
+
+    /** find the vertex with the maximum out-degree **/
+    val maxOutDegreeVertex = graph.outDegrees.max(1).map(in => in._1)
+
+    /** find the vertex with the minimum out-degree **/
+    val minOutDegreeVertex = graph.outDegrees.min(1).map(in => in._1)
+
+    /** print the results **/
+    env.fromElements(numVertices).printOnTaskManager("Total number of vertices")
+    env.fromElements(numEdges).printOnTaskManager("Total number of edges")
+    avgDegree.printOnTaskManager("Average node degree")
+    maxInDegreeVertex.printOnTaskManager("Vertex with Max in-degree")
+    minInDegreeVertex.printOnTaskManager("Vertex with Max in-degree")
+    maxOutDegreeVertex.printOnTaskManager("Vertex with Max out-degree")
+    minOutDegreeVertex.printOnTaskManager("Vertex with Max out-degree")
+
+  }
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      fileOutput = true
+      if (args.length == 1) {
+        edgesPath = args(0)
+        true
+      } else {
+        System.err.println("Usage: GraphMetrics <edges path>")
+        false
+      }
+    } else {
+      System.out.println("Executing GraphMetrics example with built-in default data.")
+      System.out.println("  Provide parameters to read input data from a file.")
+      System.out.println("  Usage: GraphMetrics <edges path>")
+      true
+    }
+  }
+
+  private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, NullValue]] = {
+    if (fileOutput) {
+      env.readCsvFile[(Long, Long)](
+        edgesPath,
+        fieldDelimiter = "\t").map(
+        in => new Edge[Long, NullValue](in._1, in._2, NullValue.getInstance()))
+    } else {
+      env.generateSequence(1, numVertices).flatMap[Edge[Long, NullValue]](
+        (key: Long, out: Collector[Edge[Long, NullValue]]) => {
+          val numOutEdges: Int = (Math.random() * (numVertices / 2)).toInt
+          for ( i <- 0 to numOutEdges ) {
+            var target: Long = ((Math.random() * numVertices) + 1).toLong
+            new Edge[Long, NullValue](key, target, NullValue.getInstance())
+          }
+      })
+    }
+  }
+
+  private var fileOutput: Boolean = false
+  private var edgesPath: String = null
+  private var outputPath: String = null
+  private val numVertices = 100
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
new file mode 100644
index 0000000..65a8e7f
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
@@ -0,0 +1,170 @@
+/*
+ * 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.scala.example;
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.types.NullValue
+import org.apache.flink.graph.Edge
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.spargel.VertexUpdateFunction
+import org.apache.flink.graph.spargel.MessageIterator
+import org.apache.flink.graph.Vertex
+import org.apache.flink.graph.spargel.MessagingFunction
+import scala.collection.JavaConversions._
+import org.apache.flink.graph.scala.utils.Tuple3ToEdgeMap
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData
+
+/**
+ * This example shows how to use Gelly's vertex-centric iterations.
+ * 
+ * It is an implementation of the Single-Source-Shortest-Paths algorithm. 
+ *
+ * The input file is a plain text file and must be formatted as follows:
+ * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
+ * separated by tabs. Edges themselves are separated by newlines.
+ * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
+ * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
+ *
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.graph.example.utils.SingleSourceShortestPathsData]]
+ */
+object SingleSourceShortestPaths {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edges: DataSet[Edge[Long, Double]] = getEdgesDataSet(env)
+    val graph = Graph.fromDataSet[Long, Double, Double](edges, new InitVertices(srcVertexId), env)
+
+    // Execute the vertex-centric iteration
+    val result = graph.runVertexCentricIteration(new VertexDistanceUpdater,
+      new MinDistanceMessenger, maxIterations)
+
+    // Extract the vertices as the result
+    val singleSourceShortestPaths = result.getVertices
+
+    // emit result
+    if (fileOutput) {
+      singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",")
+      env.execute("Single Source Shortest Paths Example")
+    } else {
+      singleSourceShortestPaths.print()
+    }
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //  Single Source Shortest Path UDFs
+  // --------------------------------------------------------------------------------------------
+
+  private final class InitVertices(srcId: Long) extends MapFunction[Long, Double] {
+
+    override def map(id: Long) = {
+      if (id.equals(srcId)) {
+        0.0
+      } else {
+        Double.PositiveInfinity
+      }
+    }
+  }
+
+  /**
+   * Function that updates the value of a vertex by picking the minimum
+   * distance from all incoming messages.
+   */
+  private final class VertexDistanceUpdater extends VertexUpdateFunction[Long, Double, Double] {
+
+    override def updateVertex(vertex: Vertex[Long, Double], inMessages: MessageIterator[Double]) {
+      var minDistance = Double.MaxValue
+      while (inMessages.hasNext) {
+        var msg = inMessages.next
+        if (msg < minDistance) {
+          minDistance = msg
+        }
+      }
+      if (vertex.getValue > minDistance) {
+        setNewVertexValue(minDistance)
+      }
+    }
+  }
+
+  /**
+   * Distributes the minimum distance associated with a given vertex among all
+   * the target vertices summed up with the edge's value.
+   */
+  private final class MinDistanceMessenger extends
+    MessagingFunction[Long, Double, Double, Double] {
+
+    override def sendMessages(vertex: Vertex[Long, Double]) {
+      for (edge: Edge[Long, Double] <- getEdges) {
+        sendMessageTo(edge.getTarget(), vertex.getValue + edge.getValue)
+      }
+    }
+  }
+
+  // ****************************************************************************
+  // UTIL METHODS
+  // ****************************************************************************
+
+  private var fileOutput = false
+  private var srcVertexId = 1L
+  private var edgesInputPath: String = null
+  private var outputPath: String = null
+  private var maxIterations = 5
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if(args.length > 0) {
+      if(args.length != 4) {
+        System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+          " <input edges path> <output path> <num iterations>")
+        false
+      }
+      fileOutput = true
+      srcVertexId = args(0).toLong
+      edgesInputPath = args(1)
+      outputPath = args(2)
+      maxIterations = (3).toInt
+    } else {
+      System.out.println("Executing Single Source Shortest Paths example "
+        + "with default parameters and built-in default data.")
+      System.out.println("  Provide parameters to read input data from files.")
+      System.out.println("  See the documentation for the correct format of input files.")
+      System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+        " <input edges path> <output path> <num iterations>");
+    }
+    true
+  }
+
+  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, Double]] = {
+    if (fileOutput) {
+      env.readCsvFile[(Long, Long, Double)](edgesInputPath,
+        lineDelimiter = "\n",
+        fieldDelimiter = "\t")
+        .map(new Tuple3ToEdgeMap[Long, Double]())
+    } else {
+      val edgeData = SingleSourceShortestPathsData.DEFAULT_EDGES map {
+        case Array(x, y, z) => (x.asInstanceOf[Long], y.asInstanceOf[Long],
+          z.asInstanceOf[Double])
+      }
+      env.fromCollection(edgeData).map(new Tuple3ToEdgeMap[Long, Double]())
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
new file mode 100644
index 0000000..159a100
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
@@ -0,0 +1,30 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.graph.{Graph => JGraph}
+
+import _root_.scala.reflect.ClassTag
+
+
+package object scala {
+  private[flink] def wrapGraph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag,
+  EV: TypeInformation : ClassTag](javagraph: JGraph[K, VV, EV]) = new Graph[K, VV, EV](javagraph)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
new file mode 100644
index 0000000..909dbb4
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.scala.utils
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.Edge
+
+@SerialVersionUID(1L)
+class EdgeToTuple3Map[K, EV] extends MapFunction[Edge[K, EV], (K, K, EV)] {
+
+  override def map(value: Edge[K, EV]): (K, K, EV) = {
+    (value.getSource, value.getTarget, value.getValue)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
new file mode 100644
index 0000000..fd6b8c5
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.scala.utils
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.Vertex
+
+@SerialVersionUID(1L)
+class Tuple2ToVertexMap[K, VV] extends MapFunction[(K, VV), Vertex[K, VV]] {
+
+  override def map(value: (K, VV)): Vertex[K, VV] = {
+    new Vertex(value._1, value._2)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
new file mode 100644
index 0000000..d0e07cc
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.scala.utils
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.Edge
+
+@SerialVersionUID(1L)
+class Tuple3ToEdgeMap[K, EV] extends MapFunction[(K, K, EV), Edge[K, EV]] {
+
+  override def map(value: (K, K, EV)): Edge[K, EV] = {
+    new Edge(value._1, value._2, value._3)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
new file mode 100644
index 0000000..faf4e10
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.scala.utils
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.Vertex
+
+@SerialVersionUID(1L)
+class VertexToTuple2Map[K, VV] extends MapFunction[Vertex[K, VV], (K, VV)] {
+
+  override def map(value: Vertex[K, VV]): (K, VV) = {
+    (value.getId, value.getValue)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
new file mode 100644
index 0000000..55faee3
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming.api.scala
+
+import java.lang.reflect.Method
+import org.apache.flink.graph.scala._
+import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
+import org.apache.flink.graph.{Graph => JavaGraph}
+import scala.language.existentials
+import org.junit.Test
+
+/**
+ * This checks whether the Gelly Scala API is up to feature parity with the Java API.
+ * Implements the {@link ScalaAPICompletenessTest} for Gelly.
+ */
+class GellyScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
+
+  override def isExcludedByName(method: Method): Boolean = {
+    val name = method.getDeclaringClass.getName + "." + method.getName
+    val excludedNames = Seq("org.apache.flink.graph.Graph.getContext")
+    excludedNames.contains(name)
+  }
+
+  @Test
+  override def testCompleteness(): Unit = {
+    checkMethods("Graph", "Graph", classOf[JavaGraph[_, _, _]], classOf[Graph[_, _, _]])
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
new file mode 100644
index 0000000..1c2cf54
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
@@ -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.scala.test
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.{Edge, Vertex}
+
+object TestGraphUtils {
+
+    def getLongLongVertexData(env: ExecutionEnvironment): DataSet[Vertex[Long, Long]] = {
+        return env.fromCollection(getLongLongVertices)
+    }
+
+    def getLongLongEdgeData(env: ExecutionEnvironment): DataSet[Edge[Long, Long]] = {
+        return env.fromCollection(getLongLongEdges)
+    }
+
+    def getLongLongVertices: List[Vertex[Long, Long]] = {
+        List(
+            new Vertex[Long, Long](1L, 1L),
+            new Vertex[Long, Long](2L, 2L),
+            new Vertex[Long, Long](3L, 3L),
+            new Vertex[Long, Long](4L, 4L),
+            new Vertex[Long, Long](5L, 5L)
+        )
+    }
+
+    def getLongLongEdges: List[Edge[Long, Long]] = {
+        List(
+            new Edge[Long, Long](1L, 2L, 12L),
+            new Edge[Long, Long](1L, 3L, 13L),
+            new Edge[Long, Long](2L, 3L, 23L),
+            new Edge[Long, Long](3L, 4L, 34L),
+            new Edge[Long, Long](3L, 5L, 35L),
+            new Edge[Long, Long](4L, 5L, 45L),
+            new Edge[Long, Long](5L, 1L, 51L)
+        )
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
new file mode 100644
index 0000000..b347049
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class DegreesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testInDegrees {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.inDegrees.collect().toList
+    expectedResult = "(1,1)\n" + "(2,1)\n" + "(3,2)\n" + "(4,1)\n" + "(5,2)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testOutDegrees {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.outDegrees.collect().toList
+    expectedResult = "(1,2)\n" + "(2,1)\n" + "(3,2)\n" + "(4,1)\n" + "(5,1)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testGetDegrees {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.getDegrees.collect().toList
+    expectedResult = "(1,3)\n" + "(2,2)\n" + "(3,4)\n" + "(4,2)\n" + "(5,3)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
new file mode 100644
index 0000000..6ceaf16
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
@@ -0,0 +1,225 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+import java.io.IOException
+import org.apache.flink.core.fs.FileInputSplit
+import java.io.File
+import java.io.OutputStreamWriter
+import java.io.FileOutputStream
+import java.io.FileOutputStream
+import com.google.common.base.Charsets
+import org.apache.flink.core.fs.Path
+import org.apache.flink.types.NullValue
+import org.apache.flink.api.common.functions.MapFunction
+
+@RunWith(classOf[Parameterized])
+class GraphCreationWithCsvITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvWithValues {
+    /*
+     * Test with two Csv files, both vertices and edges have values
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val verticesContent =  "1,1\n2,2\n3,3\n"
+    val verticesSplit = createTempFile(verticesContent)
+    val edgesContent =  "1,2,ot\n3,2,tt\n3,1,to\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, Long, String](
+        readVertices = true,
+        pathVertices = verticesSplit.getPath.toString,
+        pathEdges = edgesSplit.getPath.toString,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvNoEdgeValues {
+    /*
+     * Test with two Csv files; edges have no values
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val verticesContent =  "1,one\n2,two\n3,three\n"
+    val verticesSplit = createTempFile(verticesContent)
+    val edgesContent =  "1,2\n3,2\n3,1\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, String, NullValue](
+        readVertices = true,
+        pathVertices = verticesSplit.getPath.toString,
+        pathEdges = edgesSplit.getPath.toString,
+        hasEdgeValues = false,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,one,two,(null)\n3,2,three,two,(null)\n3,1,three,one,(null)\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvWithMapperValues {
+    /*
+     * Test with edges Csv file and vertex mapper initializer
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edgesContent =  "1,2,12\n3,2,32\n3,1,31\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, Double, Long](
+        readVertices = false,
+        pathEdges = edgesSplit.getPath.toString,
+        mapper = new VertexDoubleIdAssigner(),
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,1.0,2.0,12\n3,2,3.0,2.0,32\n3,1,3.0,1.0,31\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvNoVertexValues {
+    /*
+     * Test with edges Csv file: no vertex values
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edgesContent =  "1,2,12\n3,2,32\n3,1,31\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, NullValue, Long](
+        readVertices = false,
+        pathEdges = edgesSplit.getPath.toString,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,(null),(null),12\n3,2,(null),(null),32\n" +
+      "3,1,(null),(null),31\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvNoValues {
+    /*
+     * Test with edges Csv file: neither vertex nor edge values
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edgesContent =  "1,2\n3,2\n3,1\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, NullValue, NullValue](
+        readVertices = false,
+        pathEdges = edgesSplit.getPath.toString,
+        hasEdgeValues = false,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,(null),(null),(null)\n" +
+      "3,2,(null),(null),(null)\n3,1,(null),(null),(null)\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvOptionsVertices {
+    /*
+     * Test the options for vertices: delimiters, comments, ignore first line.
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val verticesContent =  "42#42\t" + "%this-is-a-comment\t" +
+      "1#1\t" + "2#2\t" + "3#3\t"
+    val verticesSplit = createTempFile(verticesContent)
+    val edgesContent =  "1,2,ot\n3,2,tt\n3,1,to\n"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, Long, String](
+        readVertices = true,
+        pathVertices = verticesSplit.getPath.toString,
+        lineDelimiterVertices = "\t",
+        fieldDelimiterVertices = "#",
+        ignoreFirstLineVertices = true,
+        ignoreCommentsVertices = "%",
+        pathEdges = edgesSplit.getPath.toString,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testCsvOptionsEdges {
+    /*
+     * Test the options for edges: delimiters, comments, ignore first line.
+     */
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val verticesContent =  "1,1\n2,2\n3,3\n"
+    val verticesSplit = createTempFile(verticesContent)
+    val edgesContent =  "42#42#ignore&" + "1#2#ot&" + "3#2#tt&" + "3#1#to&" +
+      "//this-is-a-comment"
+    val edgesSplit = createTempFile(edgesContent)
+    val graph = Graph.fromCsvReader[Long, Long, String](
+        pathVertices = verticesSplit.getPath.toString,
+        readVertices = true,
+        lineDelimiterEdges = "&",
+        fieldDelimiterEdges = "#",
+        ignoreFirstLineEdges = true,
+        ignoreCommentsEdges = "//",
+        pathEdges = edgesSplit.getPath.toString,
+        env = env)
+    
+    val result = graph.getTriplets.collect()
+    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
+    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
+  }
+
+  @throws(classOf[IOException])
+  def createTempFile(content: String): FileInputSplit = {
+    val tempFile = File.createTempFile("test_contents", "tmp")
+    tempFile.deleteOnExit()
+
+    val wrt = new OutputStreamWriter(new FileOutputStream(tempFile), Charsets.UTF_8)
+    wrt.write(content)
+    wrt.close()
+
+    new FileInputSplit(0, new Path(tempFile.toURI.toString), 0, tempFile.length,
+        Array("localhost"));
+    }
+
+    final class VertexDoubleIdAssigner extends MapFunction[Long, Double] {
+      @throws(classOf[Exception])
+      def map(id: Long): Double = {id.toDouble}
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
new file mode 100644
index 0000000..4b776e2
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
@@ -0,0 +1,260 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.{Edge, Vertex}
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class GraphMutationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddVertex {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+
+    val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L))
+    val res = newgraph.getVertices.collect().toList
+    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddVertexExisting {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addVertex(new Vertex[Long, Long](1L, 1L))
+    val res = newgraph.getVertices.collect().toList
+    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddVertexNoEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L))
+    val res = newgraph.getVertices.collect().toList
+    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+
+    val newgraph = graph.addVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](6L, 6L),
+        new Vertex[Long, Long](7L, 7L)))
+    val res = newgraph.getVertices.collect().toList
+    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n" + "7,7\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddVerticesExisting {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+
+    val newgraph = graph.addVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](5L, 5L),
+        new Vertex[Long, Long](6L, 6L)))
+    val res = newgraph.getVertices.collect().toList
+    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveVertex {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeVertex(new Vertex[Long, Long](5L, 5L))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveInvalidVertex {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeVertex(new Vertex[Long, Long](6L, 6L))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
+      "45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](1L, 1L),
+        new Vertex[Long, Long](2L, 2L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveValidAndInvalidVertex {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](1L, 1L),
+        new Vertex[Long, Long](6L, 6L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddEdge {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addEdge(new Vertex[Long, Long](6L, 6L), new Vertex[Long, Long](1L,
+      1L), 61L)
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
+      "45\n" + "5,1,51\n" + "6,1,61\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addEdges(List[Edge[Long, Long]](new Edge(2L, 4L, 24L),
+       new Edge(4L, 1L, 41L), new Edge(4L, 3L, 43L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "2,4,24\n" + "3,4,34\n" + "3,5," +
+    "35\n" + "4,1,41\n" + "4,3,43\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddEdgesInvalidVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addEdges(List[Edge[Long, Long]](new Edge(6L, 1L, 61L),
+       new Edge(7L, 8L, 78L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5," +
+    "35\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testAddExistingEdge {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.addEdge(new Vertex[Long, Long](1L, 1L), new Vertex[Long, Long](2L,
+      2L), 12L)
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5," +
+      "35\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveEdge {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeEdge(new Edge[Long, Long](5L, 1L, 51L))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveInvalidEdge {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeEdge(new Edge[Long, Long](6L, 1L, 61L))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
+      "45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeEdges(List[Edge[Long, Long]](new Edge(1L, 2L, 12L),
+      new Edge(4L, 5L, 45L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testRemoveSameEdgeTwiceEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val newgraph = graph.removeEdges(List[Edge[Long, Long]](new Edge(1L, 2L, 12L),
+       new Edge(1L, 2L, 12L)))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
new file mode 100644
index 0000000..7f7ebc0
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
@@ -0,0 +1,282 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.common.functions.FilterFunction
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.{Edge, Vertex}
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class GraphOperationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+    
+  @Test
+  @throws(classOf[Exception])
+  def testUndirected {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.getUndirected.getEdges.collect().toList;
+
+    expectedResult = "1,2,12\n" + "2,1,12\n" + "1,3,13\n" + "3,1,13\n" + "2,3,23\n" + "3,2," +
+      "23\n" + "3,4,34\n" + "4,3,34\n" + "3,5,35\n" + "5,3,35\n" + "4,5,45\n" + "5,4,45\n" +
+      "5,1,51\n" + "1,5,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testReverse {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.reverse().getEdges.collect().toList;
+
+    expectedResult = "2,1,12\n" + "3,1,13\n" + "3,2,23\n" + "4,3,34\n" + "5,3,35\n" + "5,4," +
+      "45\n" + "1,5,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testSubGraph {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.subgraph(new FilterFunction[Vertex[Long, Long]] {
+      @throws(classOf[Exception])
+      def filter(vertex: Vertex[Long, Long]): Boolean = {
+        return (vertex.getValue > 2)
+      }
+    }, new FilterFunction[Edge[Long, Long]] {
+
+      @throws(classOf[Exception])
+      override def filter(edge: Edge[Long, Long]): Boolean = {
+        return (edge.getValue > 34)
+      }
+    }).getEdges.collect().toList;
+
+    expectedResult = "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testSubGraphSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.subgraph(
+      vertex => vertex.getValue > 2,
+      edge => edge.getValue > 34
+    ).getEdges.collect().toList;
+
+    expectedResult = "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testFilterOnVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.filterOnVertices(new FilterFunction[Vertex[Long, Long]] {
+      @throws(classOf[Exception])
+      def filter(vertex: Vertex[Long, Long]): Boolean = {
+        vertex.getValue > 2
+      }
+    }).getEdges.collect().toList;
+
+    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testFilterOnVerticesSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.filterOnVertices(
+      vertex => vertex.getValue > 2
+    ).getEdges.collect().toList;
+
+    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testFilterOnEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.filterOnEdges(new FilterFunction[Edge[Long, Long]] {
+      @throws(classOf[Exception])
+      def filter(edge: Edge[Long, Long]): Boolean = {
+        edge.getValue > 34
+      }
+    }).getEdges.collect().toList;
+
+    expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testFilterOnEdgesSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.filterOnEdges(
+      edge => edge.getValue > 34
+    ).getEdges.collect().toList;
+
+    expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testNumberOfVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = env.fromElements(graph.numberOfVertices).collect().toList
+    expectedResult = "5"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testNumberOfEdges {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = env.fromElements(graph.numberOfEdges).collect().toList
+    expectedResult = "7"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testVertexIds {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.getVertexIds.collect().toList
+    expectedResult = "1\n2\n3\n4\n5\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testEdgesIds {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.getEdgeIds.collect().toList
+    expectedResult = "(1,2)\n" + "(1,3)\n" + "(2,3)\n" + "(3,4)\n" + "(3,5)\n" + "(4,5)\n" +
+      "(5,1)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testUnion {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
+      new Vertex[Long, Long](6L, 6L)
+    )
+    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
+      new Edge[Long, Long](6L, 1L, 61L)
+    )
+
+    val newgraph = graph.union(Graph.fromCollection(vertices, edges, env))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
+      "45\n" + "5,1,51\n" + "6,1,61\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testDifference {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
+      new Vertex[Long, Long](1L, 1L), new Vertex[Long, Long](3L, 3L),
+      new Vertex[Long, Long](6L, 6L) 
+    )
+    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
+      new Edge[Long, Long](1L, 3L, 13L), new Edge[Long, Long](1L, 6L, 16L),
+      new Edge[Long, Long](6L, 3L, 63L)
+    )
+
+    val newgraph = graph.difference(Graph.fromCollection(vertices, edges, env))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "4,5,45\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testDifferenceNoCommonVertices {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
+      new Vertex[Long, Long](6L, 6L) 
+    )
+    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
+      new Edge[Long, Long](6L, 6L, 66L)
+    )
+
+    val newgraph = graph.difference(Graph.fromCollection(vertices, edges, env))
+    val res = newgraph.getEdges.collect().toList
+    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
+      "45\n" + "5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testTriplets {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.getTriplets.collect().toList
+    expectedResult = "1,2,1,2,12\n" + "1,3,1,3,13\n" + "2,3,2,3,23\n" + "3,4,3,4,34\n" +
+      "3,5,3,5,35\n" + "4,5,4,5,45\n" + "5,1,5,1,51\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
new file mode 100644
index 0000000..3dc90fc
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
@@ -0,0 +1,151 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.Edge
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.scala.utils.EdgeToTuple3Map
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class JoinWithEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesInputDataset {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new
+        EdgeToTuple3Map[Long, Long]), new AddValuesMapper)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
+      "90\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesInputDatasetSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new
+        EdgeToTuple3Map[Long, Long]), (originalValue: Long, tupleValue: Long) =>
+      originalValue + tupleValue)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
+      "90\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesOnSource {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges
+      .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) =>
+      originalValue + tupleValue)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," +
+      "90\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesOnSourceSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges
+      .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) =>
+      originalValue + tupleValue)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," +
+      "90\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesOnTarget {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges
+      .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) =>
+      originalValue + tupleValue)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
+      "80\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithEdgesOnTargetSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges
+      .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) =>
+      originalValue + tupleValue)
+    val res = result.getEdges.collect().toList
+    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
+      "80\n" + "5,1,102\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+
+  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
+    @throws(classOf[Exception])
+    def map(tuple: (Long, Long)): Long = {
+      tuple._1 + tuple._2
+    }
+  }
+
+  final class ProjectSourceAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] {
+    @throws(classOf[Exception])
+    def map(edge: Edge[Long, Long]): (Long, Long) = {
+      (edge.getSource, edge.getValue)
+    }
+  }
+
+  final class ProjectTargetAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] {
+    @throws(classOf[Exception])
+    def map(edge: Edge[Long, Long]): (Long, Long) = {
+      (edge.getTarget, edge.getValue)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
new file mode 100644
index 0000000..98ee8b6
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
@@ -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.scala.test.operations
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.scala.utils.VertexToTuple2Map
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class JoinWithVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testJoinWithVertexSet {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result: Graph[Long, Long, Long] = graph.joinWithVertices(graph.getVertices.map(new
+        VertexToTuple2Map[Long, Long]), new AddValuesMapper)
+    val res = result.getVertices.collect().toList
+    expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testJoinWithVertexSetSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val tupleSet = graph.getVertices.map(new VertexToTuple2Map[Long, Long])
+    val result: Graph[Long, Long, Long] = graph.joinWithVertices[Long](tupleSet,
+      (originalvalue: Long, tuplevalue: Long) => originalvalue + tuplevalue)
+    val res = result.getVertices.collect().toList
+    expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+
+  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
+    @throws(classOf[Exception])
+    def map(tuple: (Long, Long)): Long = {
+      tuple._1 + tuple._2
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
new file mode 100644
index 0000000..bdfd569
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
@@ -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.scala.test.operations
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.Edge
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class MapEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithSameValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.mapEdges(new AddOneMapper).getEdges.collect().toList
+    expectedResult = "1,2,13\n" +
+      "1,3,14\n" + "" +
+      "2,3,24\n" +
+      "3,4,35\n" +
+      "3,5,36\n" +
+      "4,5,46\n" +
+      "5,1,52\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithSameValueSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.mapEdges(edge => edge.getValue + 1)
+      .getEdges.collect().toList
+    expectedResult = "1,2,13\n" +
+      "1,3,14\n" + "" +
+      "2,3,24\n" +
+      "3,4,35\n" +
+      "3,5,36\n" +
+      "4,5,46\n" +
+      "5,1,52\n"
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  final class AddOneMapper extends MapFunction[Edge[Long, Long], Long] {
+    @throws(classOf[Exception])
+    def map(edge: Edge[Long, Long]): Long = {
+      edge.getValue + 1
+    }
+  }
+
+}


[02/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
deleted file mode 100644
index 7553b32..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
+++ /dev/null
@@ -1,668 +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.operations;
-
-import java.util.Iterator;
-import java.util.List;
-
-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.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.NeighborsFunction;
-import org.apache.flink.graph.NeighborsFunctionWithVertexValue;
-import org.apache.flink.graph.ReduceNeighborsFunction;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class ReduceOnNeighborMethodsITCase extends MultipleProgramsTestBase {
-
-	public ReduceOnNeighborMethodsITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testSumOfOutNeighbors() throws Exception {
-		/*
-		 * Get the sum of out-neighbor values
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
-				graph.groupReduceOnNeighbors(new SumOutNeighbors(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-		
-		expectedResult = "1,5\n" +
-				"2,3\n" + 
-				"3,9\n" +
-				"4,5\n" + 
-				"5,1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfInNeighbors() throws Exception {
-		/*
-		 * Get the sum of in-neighbor values
-		 * times the edge weights for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSum = 
-				graph.groupReduceOnNeighbors(new SumInNeighbors(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
-		
-		expectedResult = "1,255\n" +
-				"2,12\n" + 
-				"3,59\n" +
-				"4,102\n" + 
-				"5,285\n";
-		
-		compareResultAsTuples(result, expectedResult);
-		
-		
-	}
-
-	@Test
-	public void testSumOfOAllNeighbors() throws Exception {
-		/*
-		 * Get the sum of all neighbor values
-		 * including own vertex value
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
-				graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "1,11\n" +
-				"2,6\n" + 
-				"3,15\n" +
-				"4,12\n" + 
-				"5,13\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOutNeighborsIdGreaterThanThree() throws Exception {
-		/*
-		 * Get the sum of out-neighbor values
-		 * for each vertex with id greater than three.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumOutNeighborsIdGreaterThanThree(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-		
-		expectedResult = "4,5\n" +
-				"5,1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfInNeighborsIdGreaterThanThree() throws Exception {
-		/*
-		 * Get the sum of in-neighbor values
-		 * times the edge weights for each vertex with id greater than three.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSum =
-				graph.groupReduceOnNeighbors(new SumInNeighborsIdGreaterThanThree(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
-		
-		expectedResult = "4,102\n" +
-				"5,285\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOAllNeighborsIdGreaterThanThree() throws Exception {
-		/*
-		 * Get the sum of all neighbor values
-		 * including own vertex value
-		 * for each vertex with id greater than three.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumAllNeighborsIdGreaterThanThree(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "4,12\n" +
-				"5,13\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOutNeighborsNoValue() throws Exception {
-		/*
-		 * Get the sum of out-neighbor values
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
-				graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "1,5\n" +
-				"2,3\n" + 
-				"3,9\n" +
-				"4,5\n" + 
-				"5,1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfInNeighborsNoValue() throws Exception {
-		/*
-		 * Get the sum of in-neighbor values
-		 * times the edge weights for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSum = 
-				graph.groupReduceOnNeighbors(new SumInNeighborsNoValue(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
-	
-		expectedResult = "1,255\n" +
-				"2,12\n" +
-				"3,59\n" +
-				"4,102\n" +
-				"5,285\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfAllNeighborsNoValue() throws Exception {
-		/*
-		 * Get the sum of all neighbor values
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
-				graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfAllNeighborValues.collect();
-	
-		expectedResult = "1,10\n" +
-				"2,4\n" + 
-				"3,12\n" +
-				"4,8\n" + 
-				"5,8\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
-		/*
-		 * Get the sum of out-neighbor values
-		 * for each vertex with id greater than two as well as the same sum multiplied by two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "3,9\n" +
-				"3,18\n" +
-				"4,5\n" +
-				"4,10\n" +
-				"5,1\n" +
-				"5,2";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
-		/*
-		 * Get the sum of in-neighbor values
-		 * for each vertex with id greater than two as well as the same sum multiplied by two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "3,59\n" +
-				"3,118\n" +
-				"4,204\n" +
-				"4,102\n" +
-				"5,570\n" +
-				"5,285";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
-		/*
-		 * Get the sum of all neighbor values
-		 * for each vertex with id greater than two as well as the same sum multiplied by two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
-				graph.groupReduceOnNeighbors(new SumAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfAllNeighborValues.collect();
-
-		expectedResult = "3,12\n" +
-				"3,24\n" +
-				"4,8\n" +
-				"4,16\n" +
-				"5,8\n" +
-				"5,16";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOutNeighborsMultipliedByTwo() throws Exception {
-		/*
-		 * Get the sum of out-neighbor values
-		 * for each vertex as well as the sum of out-neighbor values multiplied by two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumOutNeighborsMultipliedByTwo(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-		
-		expectedResult = "1,5\n" +
-				"1,10\n" +
-				"2,3\n" +
-				"2,6\n" +
-				"3,9\n" +
-				"3,18\n" +
-				"4,5\n" +
-				"4,10\n" +
-				"5,1\n" +
-				"5,2";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfInNeighborsSubtractOne() throws Exception {
-		/*
-		 * Get the sum of in-neighbor values
-		 * times the edge weights for each vertex as well as the same sum minus one.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSum =
-				graph.groupReduceOnNeighbors(new SumInNeighborsSubtractOne(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
-		
-		expectedResult = "1,255\n" +
-				"1,254\n" +
-				"2,12\n" +
-				"2,11\n" +
-				"3,59\n" +
-				"3,58\n" +
-				"4,102\n" +
-				"4,101\n" +
-				"5,285\n" +
-				"5,284";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testSumOfOAllNeighborsAddFive() throws Exception {
-		/*
-		 * Get the sum of all neighbor values
-		 * including own vertex value
-		 * for each vertex as well as the same sum plus five.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-				graph.groupReduceOnNeighbors(new SumAllNeighborsAddFive(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
-
-		expectedResult = "1,11\n" +
-				"1,16\n" +
-				"2,6\n" +
-				"2,11\n" +
-				"3,15\n" +
-				"3,20\n" +
-				"4,12\n" +
-				"4,17\n" +
-				"5,13\n" +
-				"5,18";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumOutNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
-	Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-				Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-				Collector<Tuple2<Long, Long>> out) throws Exception {
-			
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumInNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
-		Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-				Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-				Collector<Tuple2<Long, Long>> out) throws Exception {
-		
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumAllNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
-		Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-	
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumOutNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-					sum += neighbor.f1.getValue();
-			}
-			if(vertex.getId() > 3) {
-				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumInNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
-			}
-			if(vertex.getId() > 3) {
-				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumAllNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			if(vertex.getId() > 3) {
-				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumNeighbors implements ReduceNeighborsFunction<Long> {
-
-		@Override
-		public Long reduceNeighbors(Long firstNeighbor, Long secondNeighbor) {
-			return firstNeighbor + secondNeighbor;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumInNeighborsNoValue implements NeighborsFunction<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-			long sum = 0;
-			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
-			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
-					neighbors.iterator();
-			while(neighborsIterator.hasNext()) {
-				next = neighborsIterator.next();
-				sum += next.f2.getValue() * next.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(next.f0, sum));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
-			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
-					neighbors.iterator();
-			while(neighborsIterator.hasNext()) {
-				next = neighborsIterator.next();
-				sum += next.f2.getValue();
-			}
-			if(next.f0 > 2) {
-				out.collect(new Tuple2<Long, Long>(next.f0, sum));
-				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
-			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
-					neighbors.iterator();
-			while(neighborsIterator.hasNext()) {
-				next = neighborsIterator.next();
-				sum += next.f2.getValue() * next.f1.getValue();
-			}
-			if(next.f0 > 2) {
-				out.collect(new Tuple2<Long, Long>(next.f0, sum));
-				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
-			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
-					neighbors.iterator();
-			while(neighborsIterator.hasNext()) {
-				next = neighborsIterator.next();
-				sum += next.f2.getValue();
-			}
-			if(next.f0 > 2) {
-				out.collect(new Tuple2<Long, Long>(next.f0, sum));
-				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumOutNeighborsMultipliedByTwo implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum * 2));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumInNeighborsSubtractOne implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum - 1));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumAllNeighborsAddFive implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue() + 5));
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
deleted file mode 100644
index b32abeb..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ /dev/null
@@ -1,203 +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.operations;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.NeighborsFunctionWithVertexValue;
-import org.apache.flink.graph.ReduceNeighborsFunction;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.apache.flink.util.Collector;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-public class ReduceOnNeighborsWithExceptionITCase {
-
-	private static final int PARALLELISM = 4;
-
-	private static ForkableFlinkMiniCluster cluster;
-
-
-	@BeforeClass
-	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
-		}
-	}
-
-	/**
-	 * Test groupReduceOnNeighbors() -NeighborsFunctionWithVertexValue-
-	 * with an edge having a srcId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnNeighborsWithVVInvalidEdgeSrcId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
-
-			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test groupReduceOnNeighbors() -NeighborsFunctionWithVertexValue-
-	 * with an edge having a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnNeighborsWithVVInvalidEdgeTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
-					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
-
-			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test groupReduceOnNeighbors() -NeighborsFunction-
-	 * with an edge having a srcId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnNeighborsInvalidEdgeSrcId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
-					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
-
-			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test groupReduceOnNeighbors() -NeighborsFunction-
-	 * with an edge having a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnNeighborsInvalidEdgeTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
-					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
-
-			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumAllNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long,
-			Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateNeighbors(Vertex<Long, Long> vertex,
-									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
-									 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			long sum = 0;
-			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
-				sum += neighbor.f1.getValue();
-			}
-			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumNeighbors implements ReduceNeighborsFunction<Long> {
-
-		@Override
-		public Long reduceNeighbors(Long firstNeighbor, Long secondNeighbor) {
-			return firstNeighbor + secondNeighbor;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/pom.xml b/flink-staging/pom.xml
index 67aec5a..7bc76a7 100644
--- a/flink-staging/pom.xml
+++ b/flink-staging/pom.xml
@@ -40,12 +40,10 @@ under the License.
 		<module>flink-hadoop-compatibility</module>
 		<module>flink-streaming</module>
 		<module>flink-hbase</module>
-		<module>flink-gelly</module>
 		<module>flink-hcatalog</module>
 		<module>flink-table</module>
 		<module>flink-ml</module>
 		<module>flink-language-binding</module>
-		<module>flink-gelly-scala</module>
 		<module>flink-scala-shell</module>
 	</modules>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7e90ad6..63ffa55 100644
--- a/pom.xml
+++ b/pom.xml
@@ -63,6 +63,7 @@ under the License.
 		<module>flink-tests</module>
 		<module>flink-test-utils</module>
 		<module>flink-staging</module>
+		<module>flink-libraries</module>
 		<module>flink-quickstart</module>
 		<module>flink-contrib</module>
 		<module>flink-dist</module>


[19/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
new file mode 100644
index 0000000..e347bc5
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
@@ -0,0 +1,302 @@
+/*
+ * 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.example;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.common.ProgramDescription;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+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.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.utils.DataSetUtils;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.EdgesFunctionWithVertexValue;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.MusicProfilesData;
+import org.apache.flink.graph.library.LabelPropagation;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.util.Collector;
+
+/**
+ * This example demonstrates how to mix the DataSet Flink API with the Gelly API.
+ * The input is a set <userId - songId - playCount> triplets and
+ * a set of bad records, i.e. song ids that should not be trusted.
+ * Initially, we use the DataSet API to filter out the bad records.
+ * Then, we use Gelly to create a user -> song weighted bipartite graph and compute
+ * the top song (most listened) per user.
+ * Then, we use the DataSet API again, to create a user-user similarity graph,
+ * based on common songs, where users that are listeners of the same song
+ * are connected. A user-defined threshold on the playcount value
+ * defines when a user is considered to be a listener of a song.
+ * Finally, we use the graph API to run the label propagation community detection algorithm on
+ * the similarity graph.
+ *
+ * The triplets input is expected to be given as one triplet per line,
+ * in the following format: "<userID>\t<songID>\t<playcount>".
+ *
+ * The mismatches input file is expected to contain one mismatch record per line,
+ * in the following format:
+ * "ERROR: <songID trackID> song_title"
+ *
+ * If no arguments are provided, the example runs with default data from {@link MusicProfilesData}.
+ */
+@SuppressWarnings("serial")
+public class MusicProfiles implements ProgramDescription {
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		/**
+		 * Read the user-song-play triplets.
+		 */
+		DataSet<Tuple3<String, String, Integer>> triplets = getUserSongTripletsData(env);
+
+		/**
+		 * Read the mismatches dataset and extract the songIDs
+		 */
+		DataSet<Tuple1<String>> mismatches = getMismatchesData(env).map(new ExtractMismatchSongIds());
+
+		/**
+		 * Filter out the mismatches from the triplets dataset
+		 */
+		DataSet<Tuple3<String, String, Integer>> validTriplets = triplets
+				.coGroup(mismatches).where(1).equalTo(0)
+				.with(new FilterOutMismatches());
+
+		/**
+		 * Create a user -> song weighted bipartite graph where the edge weights
+		 * correspond to play counts
+		 */
+		Graph<String, NullValue, Integer> userSongGraph = Graph.fromTupleDataSet(validTriplets, env);
+
+		/**
+		 * Get the top track (most listened) for each user
+		 */
+		DataSet<Tuple2<String, String>> usersWithTopTrack = userSongGraph
+				.groupReduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT)
+				.filter(new FilterSongNodes());
+
+		if (fileOutput) {
+			usersWithTopTrack.writeAsCsv(topTracksOutputPath, "\n", "\t");
+		} else {
+			usersWithTopTrack.print();
+		}
+
+		/**
+		 * Create a user-user similarity graph, based on common songs, i.e. two
+		 * users that listen to the same song are connected. For each song, we
+		 * create an edge between each pair of its in-neighbors.
+		 */
+		DataSet<Edge<String, NullValue>> similarUsers = userSongGraph
+				.getEdges()
+				// filter out user-song edges that are below the playcount threshold
+				.filter(new FilterFunction<Edge<String, Integer>>() {
+					public boolean filter(Edge<String, Integer> edge) {
+						return (edge.getValue() > playcountThreshold);
+					}
+				}).groupBy(1)
+				.reduceGroup(new CreateSimilarUserEdges()).distinct();
+
+		Graph<String, Long, NullValue> similarUsersGraph = Graph.fromDataSet(similarUsers,
+				new MapFunction<String, Long>() {
+					public Long map(String value) {
+						return 1l;
+					}
+				}, env).getUndirected();
+
+		/**
+		 * Detect user communities using the label propagation library method
+		 */
+		// Initialize each vertex with a unique numeric label and run the label propagation algorithm
+		DataSet<Tuple2<String, Long>> idsWithInitialLabels = DataSetUtils
+				.zipWithUniqueId(similarUsersGraph.getVertexIds())
+				.map(new MapFunction<Tuple2<Long, String>, Tuple2<String, Long>>() {
+					@Override
+					public Tuple2<String, Long> map(Tuple2<Long, String> tuple2) throws Exception {
+						return new Tuple2<String, Long>(tuple2.f1, tuple2.f0);
+					}
+				});
+
+		DataSet<Vertex<String, Long>> verticesWithCommunity = similarUsersGraph
+				.joinWithVertices(idsWithInitialLabels,
+						new MapFunction<Tuple2<Long, Long>, Long>() {
+							public Long map(Tuple2<Long, Long> value) {
+								return value.f1;
+							}
+						}).run(new LabelPropagation<String, NullValue>(maxIterations));
+
+		if (fileOutput) {
+			verticesWithCommunity.writeAsCsv(communitiesOutputPath, "\n", "\t");
+
+			// since file sinks are lazy, we trigger the execution explicitly
+			env.execute();
+		} else {
+			verticesWithCommunity.print();
+		}
+
+	}
+
+	public static final class ExtractMismatchSongIds implements MapFunction<String, Tuple1<String>> {
+
+		public Tuple1<String> map(String value) {
+			String[] tokens = value.split("\\s+");
+			String songId = tokens[1].substring(1);
+			return new Tuple1<String>(songId);
+		}
+	}
+
+	public static final class FilterOutMismatches implements CoGroupFunction<Tuple3<String, String, Integer>,
+		Tuple1<String>, Tuple3<String, String, Integer>> {
+
+		public void coGroup(Iterable<Tuple3<String, String, Integer>> triplets,
+				Iterable<Tuple1<String>> invalidSongs, Collector<Tuple3<String, String, Integer>> out) {
+
+			if (!invalidSongs.iterator().hasNext()) {
+				// this is a valid triplet
+				for (Tuple3<String, String, Integer> triplet : triplets) {
+					out.collect(triplet);
+				}
+			}
+		}
+	}
+
+	public static final class FilterSongNodes implements FilterFunction<Tuple2<String, String>> {
+		public boolean filter(Tuple2<String, String> value) throws Exception {
+			return !value.f1.equals("");
+		}
+	}
+
+	public static final class GetTopSongPerUser	implements EdgesFunctionWithVertexValue<String, NullValue, Integer,
+		Tuple2<String, String>> {
+
+		public void iterateEdges(Vertex<String, NullValue> vertex,
+				Iterable<Edge<String, Integer>> edges, Collector<Tuple2<String, String>> out) throws Exception {
+
+			int maxPlaycount = 0;
+			String topSong = "";
+			for (Edge<String, Integer> edge : edges) {
+				if (edge.getValue() > maxPlaycount) {
+					maxPlaycount = edge.getValue();
+					topSong = edge.getTarget();
+				}
+			}
+			out.collect(new Tuple2<String, String>(vertex.getId(), topSong));
+		}
+	}
+
+	public static final class CreateSimilarUserEdges implements GroupReduceFunction<Edge<String, Integer>,
+		Edge<String, NullValue>> {
+
+		public void reduce(Iterable<Edge<String, Integer>> edges, Collector<Edge<String, NullValue>> out) {
+			List<String> listeners = new ArrayList<String>();
+			for (Edge<String, Integer> edge : edges) {
+				listeners.add(edge.getSource());
+			}
+			for (int i = 0; i < listeners.size() - 1; i++) {
+				for (int j = i + 1; j < listeners.size(); j++) {
+					out.collect(new Edge<String, NullValue>(listeners.get(i),
+							listeners.get(j), NullValue.getInstance()));
+				}
+			}
+		}
+	}
+
+	@Override
+	public String getDescription() {
+		return "Music Profiles Example";
+	}
+
+	// ******************************************************************************************************************
+	// UTIL METHODS
+	// ******************************************************************************************************************
+
+	private static boolean fileOutput = false;
+
+	private static String userSongTripletsInputPath = null;
+
+	private static String mismatchesInputPath = null;
+
+	private static String topTracksOutputPath = null;
+
+	private static int playcountThreshold = 0;
+
+	private static String communitiesOutputPath = null;
+
+	private static int maxIterations = 10;
+
+	private static boolean parseParameters(String[] args) {
+
+		if(args.length > 0) {
+			if(args.length != 6) {
+				System.err.println("Usage: MusicProfiles <input user song triplets path>" +
+						" <input song mismatches path> <output top tracks path> "
+						+ "<playcount threshold> <output communities path> <num iterations>");
+				return false;
+			}
+
+			fileOutput = true;
+			userSongTripletsInputPath = args[0];
+			mismatchesInputPath = args[1];
+			topTracksOutputPath = args[2];
+			playcountThreshold = Integer.parseInt(args[3]);
+			communitiesOutputPath = args[4];
+			maxIterations = Integer.parseInt(args[5]);
+		} else {
+			System.out.println("Executing Music Profiles example with default parameters and built-in default data.");
+			System.out.println("  Provide parameters to read input data from files.");
+			System.out.println("  See the documentation for the correct format of input files.");
+			System.out.println("Usage: MusicProfiles <input user song triplets path>" +
+					" <input song mismatches path> <output top tracks path> "
+					+ "<playcount threshold> <output communities path> <num iterations>");
+		}
+		return true;
+	}
+
+	private static DataSet<Tuple3<String, String, Integer>> getUserSongTripletsData(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(userSongTripletsInputPath)
+					.lineDelimiter("\n").fieldDelimiter("\t")
+					.types(String.class, String.class, Integer.class);
+		} else {
+			return MusicProfilesData.getUserSongTriplets(env);
+		}
+	}
+
+	private static DataSet<String> getMismatchesData(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readTextFile(mismatchesInputPath);
+		} else {
+			return MusicProfilesData.getMismatches(env);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
new file mode 100644
index 0000000..ef09bff
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
@@ -0,0 +1,199 @@
+/*
+ * 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.example;
+
+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.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
+import org.apache.flink.graph.spargel.MessageIterator;
+import org.apache.flink.graph.spargel.MessagingFunction;
+import org.apache.flink.graph.spargel.VertexUpdateFunction;
+import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
+
+/**
+ * This example shows how to use Gelly's vertex-centric iterations.
+ * 
+ * It is an implementation of the Single-Source-Shortest-Paths algorithm.
+ * For a gather-sum-apply implementation of the same algorithm, please refer to {@link GSASingleSourceShortestPaths}. 
+ *
+ * The input file is a plain text file and must be formatted as follows:
+ * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
+ * separated by tabs. Edges themselves are separated by newlines.
+ * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
+ * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
+ *
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData}
+ */
+public class SingleSourceShortestPaths implements ProgramDescription {
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
+
+		Graph<Long, Double, Double> graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env);
+
+		// Execute the vertex-centric iteration
+		Graph<Long, Double, Double> result = graph.runVertexCentricIteration(
+				new VertexDistanceUpdater(), new MinDistanceMessenger(), maxIterations);
+
+		// Extract the vertices as the result
+		DataSet<Vertex<Long, Double>> singleSourceShortestPaths = result.getVertices();
+
+		// emit result
+		if (fileOutput) {
+			singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",");
+
+			// since file sinks are lazy, we trigger the execution explicitly
+			env.execute("Single Source Shortest Paths Example");
+		} else {
+			singleSourceShortestPaths.print();
+		}
+
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Single Source Shortest Path UDFs
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("serial")
+	private static final class InitVertices implements MapFunction<Long, Double>{
+
+		private long srcId;
+
+		public InitVertices(long srcId) {
+			this.srcId = srcId;
+		}
+
+		public Double map(Long id) {
+			if (id.equals(srcId)) {
+				return 0.0;
+			}
+			else {
+				return Double.POSITIVE_INFINITY;
+			}
+		}
+	}
+
+	/**
+	 * Function that updates the value of a vertex by picking the minimum
+	 * distance from all incoming messages.
+	 */
+	@SuppressWarnings("serial")
+	public static final class VertexDistanceUpdater extends VertexUpdateFunction<Long, Double, Double> {
+
+		@Override
+		public void updateVertex(Vertex<Long, Double> vertex, MessageIterator<Double> inMessages) {
+
+			Double minDistance = Double.MAX_VALUE;
+
+			for (double msg : inMessages) {
+				if (msg < minDistance) {
+					minDistance = msg;
+				}
+			}
+
+			if (vertex.getValue() > minDistance) {
+				setNewVertexValue(minDistance);
+			}
+		}
+	}
+
+	/**
+	 * Distributes the minimum distance associated with a given vertex among all
+	 * the target vertices summed up with the edge's value.
+	 */
+	@SuppressWarnings("serial")
+	public static final class MinDistanceMessenger extends MessagingFunction<Long, Double, Double, Double> {
+
+		@Override
+		public void sendMessages(Vertex<Long, Double> vertex) {
+			for (Edge<Long, Double> edge : getEdges()) {
+				sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue());
+			}
+		}
+	}
+
+	// ******************************************************************************************************************
+	// UTIL METHODS
+	// ******************************************************************************************************************
+
+	private static boolean fileOutput = false;
+
+	private static Long srcVertexId = 1l;
+
+	private static String edgesInputPath = null;
+
+	private static String outputPath = null;
+
+	private static int maxIterations = 5;
+
+	private static boolean parseParameters(String[] args) {
+
+		if(args.length > 0) {
+			if(args.length != 4) {
+				System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+						" <input edges path> <output path> <num iterations>");
+				return false;
+			}
+
+			fileOutput = true;
+			srcVertexId = Long.parseLong(args[0]);
+			edgesInputPath = args[1];
+			outputPath = args[2];
+			maxIterations = Integer.parseInt(args[3]);
+		} else {
+				System.out.println("Executing Single Source Shortest Paths example "
+						+ "with default parameters and built-in default data.");
+				System.out.println("  Provide parameters to read input data from files.");
+				System.out.println("  See the documentation for the correct format of input files.");
+				System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
+						" <input edges path> <output path> <num iterations>");
+		}
+		return true;
+	}
+
+	private static DataSet<Edge<Long, Double>> getEdgesDataSet(ExecutionEnvironment env) {
+		if (fileOutput) {
+			return env.readCsvFile(edgesInputPath)
+					.lineDelimiter("\n")
+					.fieldDelimiter("\t")
+					.types(Long.class, Long.class, Double.class)
+					.map(new Tuple3ToEdgeMap<Long, Double>());
+		} else {
+			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
+		}
+	}
+
+	@Override
+	public String getDescription() {
+		return "Vertex-centric Single Source Shortest Paths";
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
new file mode 100644
index 0000000..c37b2b5
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
@@ -0,0 +1,95 @@
+/*
+ * 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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides the default data set used for the Simple Community Detection test program.
+ * If no parameters are given to the program, the default edge data set is used.
+ */
+public class CommunityDetectionData {
+
+	// the algorithm is not guaranteed to always converge
+	public static final Integer MAX_ITERATIONS = 30;
+
+	public static final double DELTA = 0.5f;
+
+	public static final String COMMUNITIES_SINGLE_ITERATION = "1,5\n" + "2,6\n"
+			+ "3,1\n" + "4,1\n" + "5,1\n" + "6,8\n" + "7,8\n" + "8,7"; 
+
+	public static final String COMMUNITIES_WITH_TIE = "1,2\n" + "2,1\n" + "3,1\n" + "4,1\n" + "5,1";
+
+	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
+		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
+		edges.add(new Edge<Long, Double>(2L, 3L, 4.0));
+		edges.add(new Edge<Long, Double>(2L, 4L, 5.0));
+		edges.add(new Edge<Long, Double>(3L, 5L, 6.0));
+		edges.add(new Edge<Long, Double>(5L, 6L, 7.0));
+		edges.add(new Edge<Long, Double>(5L, 7L, 8.0));
+		edges.add(new Edge<Long, Double>(6L, 7L, 9.0));
+		edges.add(new Edge<Long, Double>(7L, 12L, 10.0));
+		edges.add(new Edge<Long, Double>(8L, 9L, 11.0));
+		edges.add(new Edge<Long, Double>(8L, 10L, 12.0));
+		edges.add(new Edge<Long, Double>(8L, 11L, 13.0));
+		edges.add(new Edge<Long, Double>(9L, 10L, 14.0));
+		edges.add(new Edge<Long, Double>(9L, 11L, 15.0));
+		edges.add(new Edge<Long, Double>(10L, 11L, 16.0));
+		edges.add(new Edge<Long, Double>(10L, 12L, 17.0));
+		edges.add(new Edge<Long, Double>(11L, 12L, 18.0));
+
+		return env.fromCollection(edges);
+	}
+
+	public static DataSet<Edge<Long, Double>> getSimpleEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
+		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
+		edges.add(new Edge<Long, Double>(1L, 5L, 4.0));
+		edges.add(new Edge<Long, Double>(2L, 6L, 5.0));
+		edges.add(new Edge<Long, Double>(6L, 7L, 6.0));
+		edges.add(new Edge<Long, Double>(6L, 8L, 7.0));
+		edges.add(new Edge<Long, Double>(7L, 8L, 8.0));
+
+		return env.fromCollection(edges);
+	}
+
+	private CommunityDetectionData() {}
+
+	public static DataSet<Edge<Long, Double>> getTieEdgeDataSet(ExecutionEnvironment env) {
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 5L, 1.0));
+
+		return env.fromCollection(edges);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
new file mode 100644
index 0000000..67864eb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
@@ -0,0 +1,57 @@
+/*
+ * 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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.types.NullValue;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the connected components example program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class ConnectedComponentsDefaultData {
+
+	public static final Integer MAX_ITERATIONS = 4;
+
+	public static final String EDGES = "1	2\n" + "2	3\n" + "2	4\n" + "3	4";
+
+	public static final Object[][] DEFAULT_EDGES = new Object[][] {
+		new Object[]{1L, 2L},
+		new Object[]{2L, 3L},
+		new Object[]{2L, 4L},
+		new Object[]{3L, 4L}
+	};
+
+	public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+		List<Edge<Long, NullValue>> edgeList = new LinkedList<Edge<Long, NullValue>>();
+		for (Object[] edge : DEFAULT_EDGES) {
+			edgeList.add(new Edge<Long, NullValue>((Long) edge[0], (Long) edge[1], NullValue.getInstance()));
+		}
+		return env.fromCollection(edgeList);
+	}
+
+	public static final String VERTICES_WITH_MIN_ID = "1,1\n" + "2,1\n" + "3,1\n" + "4,1";
+
+	private ConnectedComponentsDefaultData() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
new file mode 100644
index 0000000..80765bf
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
@@ -0,0 +1,86 @@
+/*
+ * 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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.EuclideanGraphWeighing;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the Euclidean Graph example program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class EuclideanGraphData {
+
+	public static final int NUM_VERTICES = 9;
+
+	public static final String VERTICES = "1,1.0,1.0\n" + "2,2.0,2.0\n" + "3,3.0,3.0\n" + "4,4.0,4.0\n" + "5,5.0,5.0\n" +
+			"6,6.0,6.0\n" + "7,7.0,7.0\n" + "8,8.0,8.0\n" + "9,9.0,9.0";
+
+	public static DataSet<Vertex<Long, EuclideanGraphWeighing.Point>> getDefaultVertexDataSet(ExecutionEnvironment env) {
+
+		List<Vertex<Long, EuclideanGraphWeighing.Point>> vertices = new ArrayList<Vertex<Long, EuclideanGraphWeighing.Point>>();
+		for(int i=1; i<=NUM_VERTICES; i++) {
+			vertices.add(new Vertex<Long, EuclideanGraphWeighing.Point>(new Long(i),
+					new EuclideanGraphWeighing.Point(new Double(i), new Double(i))));
+		}
+
+		return env.fromCollection(vertices);
+	}
+
+	public static final String EDGES = "1,2\n" + "1,4\n" + "2,3\n" + "2,4\n" + "2,5\n" +
+			"3,5\n" + "4,5\n" + "4,6\n" + "5,7\n" + "5,9\n" + "6,7\n" + "6,8\n" +
+			"7,8\n" + "7,9\n" +  "8,9";
+
+	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, 0.0));
+		edges.add(new Edge<Long, Double>(1L, 4L, 0.0));
+		edges.add(new Edge<Long, Double>(2L, 3L, 0.0));
+		edges.add(new Edge<Long, Double>(2L, 4L, 0.0));
+		edges.add(new Edge<Long, Double>(2L, 5L, 0.0));
+		edges.add(new Edge<Long, Double>(3L, 5L, 0.0));
+		edges.add(new Edge<Long, Double>(4L, 5L, 0.0));
+		edges.add(new Edge<Long, Double>(4L, 6L, 0.0));
+		edges.add(new Edge<Long, Double>(5L, 7L, 0.0));
+		edges.add(new Edge<Long, Double>(5L, 9L, 0.0));
+		edges.add(new Edge<Long, Double>(6L, 7L, 0.0));
+		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
+		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
+		edges.add(new Edge<Long, Double>(7L, 8L, 0.0));
+		edges.add(new Edge<Long, Double>(7L, 9L, 0.0));
+		edges.add(new Edge<Long, Double>(8L, 9L, 0.0));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final String RESULTED_WEIGHTED_EDGES = "1,2,1.4142135623730951\n" + "1,4,4.242640687119285\n" +
+			"2,3,1.4142135623730951\n" + "2,4,2.8284271247461903\n" + "2,5,4.242640687119285\n" + "3,5,2.8284271247461903\n" +
+			"4,5,1.4142135623730951\n" + "4,6,2.8284271247461903\n" + "5,7,2.8284271247461903\n" + "5,9,5.656854249492381\n" +
+			"6,7,1.4142135623730951\n" + "6,8,2.8284271247461903\n" + "7,8,1.4142135623730951\n" + "7,9,2.8284271247461903\n" +
+			"8,9,1.4142135623730951";
+
+	private EuclideanGraphData() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
new file mode 100644
index 0000000..7fbee46
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
@@ -0,0 +1,162 @@
+/*
+ * 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.example.utils;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.util.Collector;
+
+public class ExampleUtils {
+
+	@SuppressWarnings({ "serial", "unchecked", "rawtypes" })
+	public static void printResult(DataSet set, String msg) {
+		set.output(new PrintingOutputFormatWithMessage(msg) {
+		});
+	}
+
+	public static class PrintingOutputFormatWithMessage<T> implements
+			OutputFormat<T> {
+
+		private static final long serialVersionUID = 1L;
+
+		private transient PrintStream stream;
+
+		private transient String prefix;
+
+		private String message;
+
+		// --------------------------------------------------------------------------------------------
+
+		/**
+		 * Instantiates a printing output format that prints to standard out.
+		 */
+		public PrintingOutputFormatWithMessage() {
+		}
+
+		public PrintingOutputFormatWithMessage(String msg) {
+			this.message = msg;
+		}
+
+		@Override
+		public void open(int taskNumber, int numTasks) {
+			// get the target stream
+			this.stream = System.out;
+
+			// set the prefix to message
+			this.prefix = message + ": ";
+		}
+
+		@Override
+		public void writeRecord(T record) {
+			if (this.prefix != null) {
+				this.stream.println(this.prefix + record.toString());
+			} else {
+				this.stream.println(record.toString());
+			}
+		}
+
+		@Override
+		public void close() {
+			this.stream = null;
+			this.prefix = null;
+		}
+
+		@Override
+		public String toString() {
+			return "Print to System.out";
+		}
+
+		@Override
+		public void configure(Configuration parameters) {
+		}
+	}
+
+	@SuppressWarnings("serial")
+	public static DataSet<Vertex<Long, NullValue>> getVertexIds(
+			ExecutionEnvironment env, final long numVertices) {
+		return env.generateSequence(1, numVertices).map(
+				new MapFunction<Long, Vertex<Long, NullValue>>() {
+					public Vertex<Long, NullValue> map(Long l) {
+						return new Vertex<Long, NullValue>(l, NullValue
+								.getInstance());
+					}
+				});
+	}
+
+	@SuppressWarnings("serial")
+	public static DataSet<Edge<Long, NullValue>> getRandomEdges(
+			ExecutionEnvironment env, final long numVertices) {
+		return env.generateSequence(1, numVertices).flatMap(
+				new FlatMapFunction<Long, Edge<Long, NullValue>>() {
+					@Override
+					public void flatMap(Long key, Collector<Edge<Long, NullValue>> out) throws Exception {
+						int numOutEdges = (int) (Math.random() * (numVertices / 2));
+						for (int i = 0; i < numOutEdges; i++) {
+							long target = (long) (Math.random() * numVertices) + 1;
+							out.collect(new Edge<Long, NullValue>(key, target,
+									NullValue.getInstance()));
+						}
+					}
+				});
+	}
+
+	public static DataSet<Vertex<Long, Double>> getLongDoubleVertexData(
+			ExecutionEnvironment env) {
+		List<Vertex<Long, Double>> vertices = new ArrayList<Vertex<Long, Double>>();
+		vertices.add(new Vertex<Long, Double>(1L, 1.0));
+		vertices.add(new Vertex<Long, Double>(2L, 2.0));
+		vertices.add(new Vertex<Long, Double>(3L, 3.0));
+		vertices.add(new Vertex<Long, Double>(4L, 4.0));
+		vertices.add(new Vertex<Long, Double>(5L, 5.0));
+
+		return env.fromCollection(vertices);
+	}
+
+	public static DataSet<Edge<Long, Double>> getLongDoubleEdgeData(
+			ExecutionEnvironment env) {
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, 12.0));
+		edges.add(new Edge<Long, Double>(1L, 3L, 13.0));
+		edges.add(new Edge<Long, Double>(2L, 3L, 23.0));
+		edges.add(new Edge<Long, Double>(3L, 4L, 34.0));
+		edges.add(new Edge<Long, Double>(3L, 5L, 35.0));
+		edges.add(new Edge<Long, Double>(4L, 5L, 45.0));
+		edges.add(new Edge<Long, Double>(5L, 1L, 51.0));
+
+		return env.fromCollection(edges);
+	}
+
+	/**
+	 * Private constructor to prevent instantiation.
+	 */
+	private ExampleUtils() {
+		throw new RuntimeException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
new file mode 100644
index 0000000..7b69ec0
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
@@ -0,0 +1,95 @@
+/*
+ * 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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Vertex;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the IncrementalSSSP example program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class IncrementalSSSPData {
+
+	public static final int NUM_VERTICES = 5;
+
+	public static final String VERTICES = "1,6.0\n" + "2,2.0\n" + "3,3.0\n" + "4,1.0\n" + "5,0.0";
+
+	public static DataSet<Vertex<Long, Double>> getDefaultVertexDataSet(ExecutionEnvironment env) {
+
+		List<Vertex<Long, Double>> vertices = new ArrayList<Vertex<Long, Double>>();
+		vertices.add(new Vertex<Long, Double>(1L, 6.0));
+		vertices.add(new Vertex<Long, Double>(2L, 2.0));
+		vertices.add(new Vertex<Long, Double>(3L, 3.0));
+		vertices.add(new Vertex<Long, Double>(4L, 1.0));
+		vertices.add(new Vertex<Long, Double>(5L, 0.0));
+
+		return env.fromCollection(vertices);
+	}
+
+	public static final String EDGES = "1,3,3.0\n" + "2,4,3.0\n" + "2,5,2.0\n" + "3,2,1.0\n" + "3,5,5.0\n" +
+			"4,5,1.0";
+
+	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
+		edges.add(new Edge<Long, Double>(2L, 4L, 3.0));
+		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
+		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(3L, 5L, 5.0));
+		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final String EDGES_IN_SSSP = "1,3,3.0\n" + "2,5,2.0\n" + "3,2,1.0\n" + "4,5,1.0";
+
+	public static final DataSet<Edge<Long, Double>> getDefaultEdgesInSSSP(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
+		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
+		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final String SRC_EDGE_TO_BE_REMOVED = "2";
+
+	public static final String TRG_EDGE_TO_BE_REMOVED = "5";
+
+	public static final String VAL_EDGE_TO_BE_REMOVED = "2.0";
+
+	public static final Edge<Long, Double> getDefaultEdgeToBeRemoved() {
+
+		return new Edge<Long, Double>(2L, 5L, 2.0);
+	}
+
+	public static final String RESULTED_VERTICES = "1," + Double.MAX_VALUE + "\n" + "2," + Double.MAX_VALUE+ "\n"
+			+ "3," + Double.MAX_VALUE + "\n" + "4,1.0\n" + "5,0.0";
+
+	private IncrementalSSSPData() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
new file mode 100644
index 0000000..7564b95
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
@@ -0,0 +1,58 @@
+/*
+ * 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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the Jaccard Similarity Measure example program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class JaccardSimilarityMeasureData {
+
+	public static final String EDGES = "1	2\n" + "1	3\n" + "1	4\n" + "1	5\n" + "2	3\n" + "2	4\n" +
+			"2	5\n" + "3	4\n" + "3	5\n" + "4	5";
+
+	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(1L, 2L, new Double(0)));
+		edges.add(new Edge<Long, Double>(1L, 3L, new Double(0)));
+		edges.add(new Edge<Long, Double>(1L, 4L, new Double(0)));
+		edges.add(new Edge<Long, Double>(1L, 5L, new Double(0)));
+		edges.add(new Edge<Long, Double>(2L, 3L, new Double(0)));
+		edges.add(new Edge<Long, Double>(2L, 4L, new Double(0)));
+		edges.add(new Edge<Long, Double>(2L, 5L, new Double(0)));
+		edges.add(new Edge<Long, Double>(3L, 4L, new Double(0)));
+		edges.add(new Edge<Long, Double>(3L, 5L, new Double(0)));
+		edges.add(new Edge<Long, Double>(4L, 5L, new Double(0)));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final String JACCARD_EDGES = "1,2,0.6\n" + "1,3,0.6\n" + "1,4,0.6\n" + "1,5,0.6\n" +
+			"2,3,0.6\n" + "2,4,0.6\n" + "2,5,0.6\n" + "3,4,0.6\n" + "3,5,0.6\n" + "4,5,0.6";
+
+	private JaccardSimilarityMeasureData() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
new file mode 100644
index 0000000..0a92097
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
@@ -0,0 +1,114 @@
+/*
+ * 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.example.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.NullValue;
+
+/**
+ * Provides the default data set used for the Label Propagation test program.
+ * If no parameters are given to the program, the default edge data set is used.
+ */
+public class LabelPropagationData {
+	
+	public static final String LABELS_AFTER_1_ITERATION = "1,10\n" +
+			"2,10\n" +
+			"3,10\n" +
+			"4,40\n" +
+			"5,40\n" +
+			"6,40\n" +
+			"7,40\n";
+
+	public static final String LABELS_WITH_TIE ="1,10\n" +
+			"2,10\n" +
+			"3,10\n" +
+			"4,10\n" +
+			"5,20\n" +
+			"6,20\n" +
+			"7,20\n" +
+			"8,20\n" +
+			"9,20\n";
+
+	private LabelPropagationData() {}
+
+	public static final DataSet<Vertex<Long, Long>> getDefaultVertexSet(ExecutionEnvironment env) {
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(1l, 10l));
+		vertices.add(new Vertex<Long, Long>(2l, 10l));
+		vertices.add(new Vertex<Long, Long>(3l, 30l));
+		vertices.add(new Vertex<Long, Long>(4l, 40l));
+		vertices.add(new Vertex<Long, Long>(5l, 40l));
+		vertices.add(new Vertex<Long, Long>(6l, 40l));
+		vertices.add(new Vertex<Long, Long>(7l, 40l));
+
+		return env.fromCollection(vertices);
+	}
+
+	public static final DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
+		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(4L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(5L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(7L, 3L, NullValue.getInstance()));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final DataSet<Vertex<Long, Long>> getTieVertexSet(ExecutionEnvironment env) {
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(1l, 10l));
+		vertices.add(new Vertex<Long, Long>(2l, 10l));
+		vertices.add(new Vertex<Long, Long>(3l, 10l));
+		vertices.add(new Vertex<Long, Long>(4l, 10l));
+		vertices.add(new Vertex<Long, Long>(5l, 0l));
+		vertices.add(new Vertex<Long, Long>(6l, 20l));
+		vertices.add(new Vertex<Long, Long>(7l, 20l));
+		vertices.add(new Vertex<Long, Long>(8l, 20l));
+		vertices.add(new Vertex<Long, Long>(9l, 20l));
+
+		return env.fromCollection(vertices);
+	}
+
+	public static final DataSet<Edge<Long, NullValue>> getTieEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
+		edges.add(new Edge<Long, NullValue>(1L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(2L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(5L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(6L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(7L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(8L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(9L, 5L, NullValue.getInstance()));
+
+		return env.fromCollection(edges);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
new file mode 100644
index 0000000..3a97d1f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
@@ -0,0 +1,108 @@
+/*
+ * 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.example.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+
+/**
+ * Provides the default data sets used for the Music Profiles example program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class MusicProfilesData {
+
+	public static DataSet<Tuple3<String, String, Integer>> getUserSongTriplets(ExecutionEnvironment env) {
+		List<Tuple3<String, String, Integer>> triplets = new ArrayList<Tuple3<String, String, Integer>>();
+		
+		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_1", 100));
+		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_2", 10));
+		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_3", 20));
+		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_4", 30));
+		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_5", 1));
+		
+		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_6", 40));
+		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_7", 10));
+		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_8", 3));
+		
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_1", 100));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_2", 10));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_3", 20));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_8", 30));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_9", 1));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_10", 8));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_11", 90));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_12", 30));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_13", 34));
+		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_14", 17));
+		
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_1", 100));
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_6", 10));
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_8", 20));
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_12", 30));
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_13", 1));
+		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_15", 1));
+		
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_3", 300));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_4", 4));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_5", 5));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_8", 8));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_9", 9));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_10", 10));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_12", 12));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_13", 13));
+		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_15", 15));
+
+		triplets.add(new Tuple3<String, String, Integer>("user_6", "song_6", 30));
+
+		return env.fromCollection(triplets);
+	}
+	
+	public static DataSet<String> getMismatches(ExecutionEnvironment env) {
+		List<String> errors = new ArrayList<String>();
+		errors.add("ERROR: <song_8 track_8> Sever");
+		errors.add("ERROR: <song_15 track_15> Black Trees");
+		return env.fromCollection(errors);
+	}
+
+	public static final String USER_SONG_TRIPLETS = "user_1	song_1	100\n" + "user_1	song_5	200\n"
+			+ "user_2	song_1	10\n" + "user_2	song_4	20\n"
+			+ "user_3	song_2	3\n"
+			+ "user_4	song_2	1\n" + "user_4	song_3	2\n"
+			+ "user_5	song_3	30";
+
+	public static final String MISMATCHES = "ERROR: <song_5 track_8> Angie";
+
+	public static final String MAX_ITERATIONS = "2";
+
+	public static final String TOP_SONGS_RESULT = "user_1	song_1\n" +
+								"user_2	song_4\n" +
+								"user_3	song_2\n" +
+								"user_4	song_3\n" +
+								"user_5	song_3";
+
+	public static final String COMMUNITIES_RESULT = "user_1	1\n" +
+								"user_2	1\n" +
+								"user_3	3\n" +
+								"user_4	3\n" +
+								"user_5	4";
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
new file mode 100644
index 0000000..58d4f8b
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
@@ -0,0 +1,69 @@
+/*
+ * 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.example.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+
+/**
+ * Provides the default data set used for the PageRank test program.
+ * If no parameters are given to the program, the default edge data set is used.
+ */
+public class PageRankData {
+	
+	public static final String EDGES = "2	1\n" +
+										"5	2\n" + 
+										"5	4\n" +
+										"4	3\n" +
+										"4	2\n" +
+										"1	4\n" +
+										"1	2\n" +
+										"1	3\n" +
+										"3	5\n";
+
+	
+	public static final String RANKS_AFTER_3_ITERATIONS = "1,0.237\n" +
+														"2,0.248\n" + 
+														"3,0.173\n" +
+														"4,0.175\n" +
+														"5,0.165\n";
+
+	private PageRankData() {}
+
+	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
+		edges.add(new Edge<Long, Double>(2L, 1L, 1.0));
+		edges.add(new Edge<Long, Double>(5L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(5L, 4L, 1.0));
+		edges.add(new Edge<Long, Double>(4L, 3L, 1.0));
+		edges.add(new Edge<Long, Double>(4L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
+		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
+		edges.add(new Edge<Long, Double>(3L, 5L, 1.0));
+
+		return env.fromCollection(edges);
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
new file mode 100644
index 0000000..6b985c5
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
@@ -0,0 +1,62 @@
+/*
+ * 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.example.utils;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+
+/**
+ * Provides the default data set used for the Single Source Shortest Paths example program.
+ * If no parameters are given to the program, the default edge data set is used.
+ */
+public class SingleSourceShortestPathsData {
+
+	public static final Long SRC_VERTEX_ID = 1L;
+
+	public static final String EDGES = "1\t2\t12.0\n" + "1\t3\t13.0\n" + "2\t3\t23.0\n" + "3\t4\t34.0\n" + "3\t5\t35.0\n" +
+					"4\t5\t45.0\n" + "5\t1\t51.0";
+
+	public static final Object[][] DEFAULT_EDGES = new Object[][] {
+		new Object[]{1L, 2L, 12.0},
+		new Object[]{1L, 3L, 13.0},
+		new Object[]{2L, 3L, 23.0},
+		new Object[]{3L, 4L, 34.0},
+		new Object[]{3L, 5L, 35.0},
+		new Object[]{4L, 5L, 45.0},
+		new Object[]{5L, 1L, 51.0}
+	};
+
+	public static final String RESULTED_SINGLE_SOURCE_SHORTEST_PATHS =  "1,0.0\n" + "2,12.0\n" + "3,13.0\n" + 
+								"4,47.0\n" + "5,48.0";
+
+	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+		
+		List<Edge<Long, Double>> edgeList = new LinkedList<Edge<Long, Double>>();
+		for (Object[] edge : DEFAULT_EDGES) {
+			edgeList.add(new Edge<Long, Double>((Long) edge[0], (Long) edge[1], (Double) edge[2]));
+		}
+		return env.fromCollection(edgeList);
+	}
+
+	private SingleSourceShortestPathsData() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java
new file mode 100644
index 0000000..5b2cc3d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.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.example.utils;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.types.NullValue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides the default data sets used for the Triangle Count test program.
+ * If no parameters are given to the program, the default data sets are used.
+ */
+public class TriangleCountData {
+
+	public static final String EDGES = "1	2\n"+"1	3\n"+"2	3\n"+"2	6\n"+"3	4\n"+"3	5\n"+"3	6\n"+"4	5\n"+"6	7\n";
+
+	public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
+
+		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
+		edges.add(new Edge<Long, NullValue>(1L, 2L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(2L, 6L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(3L, 4L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(3L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(3L, 6L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
+		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
+
+		return env.fromCollection(edges);
+	}
+
+	public static final String RESULTED_NUMBER_OF_TRIANGLES = "3";
+
+	private TriangleCountData () {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
new file mode 100755
index 0000000..5a8e97a
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
@@ -0,0 +1,155 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+/**
+ * The base class for the third and last step of a {@link GatherSumApplyIteration}.
+ *
+ * @param <K> the vertex ID type
+ * @param <VV> the vertex value type
+ * @param <M> the input type (produced by the Sum phase)
+ */
+@SuppressWarnings("serial")
+public abstract class ApplyFunction<K, VV, M> implements Serializable {
+
+	// --------------------------------------------------------------------------------------------
+	//  Attribute that allows access to the total number of vertices inside an iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private long numberOfVertices = -1L;
+
+	/**
+	 * Retrieves the number of vertices in the graph.
+	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * option has been set; -1 otherwise.
+	 */
+	public long getNumberOfVertices() {
+		return numberOfVertices;
+	}
+
+	void setNumberOfVertices(long numberOfVertices) {
+		this.numberOfVertices = numberOfVertices;
+	}
+
+	//---------------------------------------------------------------------------------------------
+
+	/**
+	 * This method is invoked once per superstep, after the {@link SumFunction} 
+	 * in a {@link GatherSumApplyIteration}.
+	 * It updates the Vertex values.
+	 * 
+	 * @param newValue the value computed during the current superstep.
+	 * @param currentValue the current Vertex value.
+	 */
+	public abstract void apply(M newValue, VV currentValue);
+
+	/**
+	 * Sets the result for the apply function
+	 *
+	 * @param result the result of the apply phase
+	 */
+	public void setResult(VV result) {
+		outVal.f1 = result;
+		out.collect(outVal);
+	}
+
+	/**
+	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
+	 */
+	public void preSuperstep() {}
+
+	/**
+	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
+	 */
+	public void postSuperstep() {}
+
+	/**
+	 * Gets the number of the superstep, starting at <tt>1</tt>.
+	 *
+	 * @return The number of the current superstep.
+	 */
+	public int getSuperstepNumber() {
+		return this.runtimeContext.getSuperstepNumber();
+	}
+
+	/**
+	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
+	 * all aggregates globally once per superstep and makes them available in the next superstep.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
+	 */
+	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+		return this.runtimeContext.<T>getIterationAggregator(name);
+	}
+
+	/**
+	 * Get the aggregated value that an aggregator computed in the previous iteration.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregated value of the previous iteration.
+	 */
+	public <T extends Value> T getPreviousIterationAggregate(String name) {
+		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
+	}
+
+	/**
+	 * Gets the broadcast data set registered under the given name. Broadcast data sets
+	 * are available on all parallel instances of a function.
+	 *
+	 * @param name The name under which the broadcast set is registered.
+	 * @return The broadcast data set.
+	 */
+	public <T> Collection<T> getBroadcastSet(String name) {
+		return this.runtimeContext.<T>getBroadcastVariable(name);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal methods
+	// --------------------------------------------------------------------------------------------
+
+	private IterationRuntimeContext runtimeContext;
+
+	private Collector<Vertex<K, VV>> out;
+
+	private Vertex<K, VV> outVal;
+
+	public void init(IterationRuntimeContext iterationRuntimeContext) {
+		this.runtimeContext = iterationRuntimeContext;
+	}
+
+	public void setOutput(Vertex<K, VV> vertex, Collector<Vertex<K, VV>> out) {
+		this.out = out;
+		this.outVal = vertex;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
new file mode 100644
index 0000000..8d24f16
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
@@ -0,0 +1,133 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.IterationConfiguration;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A GSAConfiguration object can be used to set the iteration name and
+ * degree of parallelism, to register aggregators and use broadcast sets in
+ * the {@link org.apache.flink.graph.gsa.GatherFunction}, {@link org.apache.flink.graph.gsa.SumFunction} as well as
+ * {@link org.apache.flink.graph.gsa.ApplyFunction}.
+ *
+ * The GSAConfiguration object is passed as an argument to
+ * {@link org.apache.flink.graph.Graph#runGatherSumApplyIteration(org.apache.flink.graph.gsa.GatherFunction,
+ * org.apache.flink.graph.gsa.SumFunction, org.apache.flink.graph.gsa.ApplyFunction, int)}
+ */
+public class GSAConfiguration extends IterationConfiguration {
+
+	/** the broadcast variables for the gather function **/
+	private List<Tuple2<String, DataSet<?>>> bcVarsGather = new ArrayList<Tuple2<String,DataSet<?>>>();
+
+	/** the broadcast variables for the sum function **/
+	private List<Tuple2<String, DataSet<?>>> bcVarsSum = new ArrayList<Tuple2<String,DataSet<?>>>();
+
+	/** the broadcast variables for the apply function **/
+	private List<Tuple2<String, DataSet<?>>> bcVarsApply = new ArrayList<Tuple2<String,DataSet<?>>>();
+
+	private EdgeDirection direction = EdgeDirection.OUT;
+
+	public GSAConfiguration() {}
+
+	/**
+	 * Adds a data set as a broadcast set to the gather function.
+	 *
+	 * @param name The name under which the broadcast data is available in the gather function.
+	 * @param data The data set to be broadcasted.
+	 */
+	public void addBroadcastSetForGatherFunction(String name, DataSet<?> data) {
+		this.bcVarsGather.add(new Tuple2<String, DataSet<?>>(name, data));
+	}
+
+	/**
+	 * Adds a data set as a broadcast set to the sum function.
+	 *
+	 * @param name The name under which the broadcast data is available in the sum function.
+	 * @param data The data set to be broadcasted.
+	 */
+	public void addBroadcastSetForSumFunction(String name, DataSet<?> data) {
+		this.bcVarsSum.add(new Tuple2<String, DataSet<?>>(name, data));
+	}
+
+	/**
+	 * Adds a data set as a broadcast set to the apply function.
+	 *
+	 * @param name The name under which the broadcast data is available in the apply function.
+	 * @param data The data set to be broadcasted.
+	 */
+	public void addBroadcastSetForApplyFunction(String name, DataSet<?> data) {
+		this.bcVarsApply.add(new Tuple2<String, DataSet<?>>(name, data));
+	}
+
+	/**
+	 * Get the broadcast variables of the GatherFunction.
+	 *
+	 * @return a List of Tuple2, where the first field is the broadcast variable name
+	 * and the second field is the broadcast DataSet.
+	 */
+	public List<Tuple2<String, DataSet<?>>> getGatherBcastVars() {
+		return this.bcVarsGather;
+	}
+
+	/**
+	 * Get the broadcast variables of the SumFunction.
+	 *
+	 * @return a List of Tuple2, where the first field is the broadcast variable name
+	 * and the second field is the broadcast DataSet.
+	 */
+	public List<Tuple2<String, DataSet<?>>> getSumBcastVars() {
+		return this.bcVarsSum;
+	}
+
+	/**
+	 * Get the broadcast variables of the ApplyFunction.
+	 *
+	 * @return a List of Tuple2, where the first field is the broadcast variable name
+	 * and the second field is the broadcast DataSet.
+	 */
+	public List<Tuple2<String, DataSet<?>>> getApplyBcastVars() {
+		return this.bcVarsApply;
+	}
+
+	/**
+	 * Gets the direction from which the neighbors are to be selected
+	 * By default the neighbors who are target of the edges are selected
+	 *
+	 * @return an EdgeDirection, which can be either IN, OUT or ALL.
+	 */
+	public EdgeDirection getDirection() {
+		return direction;
+	}
+
+	/**
+	 * Sets the direction in which neighbors are to be selected
+	 * By default the neighbors who are target of the edges are selected
+	 *
+	 * @param direction - IN, OUT or ALL
+	 */
+	public void setDirection(EdgeDirection direction) {
+		this.direction = direction;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
new file mode 100755
index 0000000..563b20e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
@@ -0,0 +1,134 @@
+/*
+ * 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.gsa;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.types.Value;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+/**
+ * The base class for the first step of a {@link GatherSumApplyIteration}.
+ * 
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ * @param <M> the output type 
+ */
+@SuppressWarnings("serial")
+public abstract class GatherFunction<VV, EV, M> implements Serializable {
+
+	// --------------------------------------------------------------------------------------------
+	//  Attribute that allows access to the total number of vertices inside an iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private long numberOfVertices = -1L;
+
+	/**
+	 * Retrieves the number of vertices in the graph.
+	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * option has been set; -1 otherwise.
+	 */
+	public long getNumberOfVertices() {
+		return numberOfVertices;
+	}
+
+	void setNumberOfVertices(long numberOfVertices) {
+		this.numberOfVertices = numberOfVertices;
+	}
+
+	//---------------------------------------------------------------------------------------------
+
+	/**
+	 * This method is invoked once per superstep, for each {@link Neighbor} of each Vertex 
+	 * in the beginning of each superstep in a {@link GatherSumApplyIteration}.
+	 * It needs to produce a partial value, which will be combined with other partial value
+	 * in the next phase of the iteration.
+	 *  
+	 * @param neighbor the input Neighbor. It provides access to the source Vertex and the Edge objects.
+	 * @return a partial result to be combined in the Sum phase.
+	 */
+	public abstract M gather(Neighbor<VV, EV> neighbor);
+
+	/**
+	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
+	 */
+	public void preSuperstep() {}
+
+	/**
+	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
+	 *
+	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
+	 */
+	public void postSuperstep() {}
+
+	/**
+	 * Gets the number of the superstep, starting at <tt>1</tt>.
+	 *
+	 * @return The number of the current superstep.
+	 */
+	public int getSuperstepNumber() {
+		return this.runtimeContext.getSuperstepNumber();
+	}
+
+	/**
+	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
+	 * all aggregates globally once per superstep and makes them available in the next superstep.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
+	 */
+	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+		return this.runtimeContext.<T>getIterationAggregator(name);
+	}
+
+	/**
+	 * Get the aggregated value that an aggregator computed in the previous iteration.
+	 *
+	 * @param name The name of the aggregator.
+	 * @return The aggregated value of the previous iteration.
+	 */
+	public <T extends Value> T getPreviousIterationAggregate(String name) {
+		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
+	}
+
+	/**
+	 * Gets the broadcast data set registered under the given name. Broadcast data sets
+	 * are available on all parallel instances of a function.
+	 *
+	 * @param name The name under which the broadcast set is registered.
+	 * @return The broadcast data set.
+	 */
+	public <T> Collection<T> getBroadcastSet(String name) {
+		return this.runtimeContext.<T>getBroadcastVariable(name);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal methods
+	// --------------------------------------------------------------------------------------------
+
+	private IterationRuntimeContext runtimeContext;
+
+	public void init(IterationRuntimeContext iterationRuntimeContext) {
+		this.runtimeContext = iterationRuntimeContext;
+	}
+}


[09/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
deleted file mode 100644
index f64c701..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphCsvReader.java
+++ /dev/null
@@ -1,486 +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;
-import com.google.common.base.Preconditions;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.io.CsvReader;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * A class to build a Graph using path(s) provided to CSV file(s) with optional vertex and edge data.
- * The class also configures the CSV readers used to read edge and vertex data such as the field types,
- * the delimiters (row and field), the fields that should be included or skipped, and other flags,
- * such as whether to skip the initial line as the header.
- * The configuration is done using the functions provided in the {@link org.apache.flink.api.java.io.CsvReader} class.
- */
-
-public class GraphCsvReader {
-
-	@SuppressWarnings("unused")
-	private final Path vertexPath, edgePath;
-	private final ExecutionEnvironment executionContext;
-	protected CsvReader edgeReader;
-	protected CsvReader vertexReader;
-	protected MapFunction<?, ?> mapper;
-	protected Class<?> vertexKey;
-	protected Class<?> vertexValue;
-	protected Class<?> edgeValue;
-
-//--------------------------------------------------------------------------------------------------------------------
-	public GraphCsvReader(Path vertexPath, Path edgePath, ExecutionEnvironment context) {
-		this.vertexPath = vertexPath;
-		this.edgePath = edgePath;
-		this.vertexReader = new CsvReader(vertexPath, context);
-		this.edgeReader = new CsvReader(edgePath, context);
-		this.mapper = null;
-		this.executionContext = context;
-	}
-
-	public GraphCsvReader(Path edgePath, ExecutionEnvironment context) {
-		this.vertexPath = null;
-		this.edgePath = edgePath;
-		this.edgeReader = new CsvReader(edgePath, context);
-		this.vertexReader = null;
-		this.mapper = null;
-		this.executionContext = context;
-	}
-
-	public <K, VV> GraphCsvReader(Path edgePath, final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-		this.vertexPath = null;
-		this.edgePath = edgePath;
-		this.edgeReader = new CsvReader(edgePath, context);
-		this.vertexReader = null;
-		this.mapper = mapper;
-		this.executionContext = context;
-	}
-
-	public GraphCsvReader (String edgePath, ExecutionEnvironment context) {
-		this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context);
-
-	}
-
-	public GraphCsvReader(String vertexPath, String edgePath, ExecutionEnvironment context) {
-		this(new Path(Preconditions.checkNotNull(vertexPath, "The file path may not be null.")),
-				new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), context);
-	}
-
-
-	public <K, VV> GraphCsvReader(String edgePath, final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-			this(new Path(Preconditions.checkNotNull(edgePath, "The file path may not be null.")), mapper, context);
-	}
-
-	/**
-	 * Creates a Graph from CSV input with vertex values and edge values.
-	 * The vertex values are specified through a vertices input file or a user-defined map function.
-	 * 
-	 * @param vertexKey the type of the vertex IDs
-	 * @param vertexValue the type of the vertex values
-	 * @param edgeValue the type of the edge values
-	 * @return a Graph with vertex and edge values.
-	 */
-	@SuppressWarnings("unchecked")
-	public <K, VV, EV> Graph<K, VV, EV> types(Class<K> vertexKey, Class<VV> vertexValue,
-			Class<EV> edgeValue) {
-
-		DataSet<Tuple2<K, VV>> vertices = null;
-
-		if (edgeReader == null) {
-			throw new RuntimeException("The edges input file cannot be null!");
-		}
-
-		DataSet<Tuple3<K, K, EV>> edges = edgeReader.types(vertexKey, vertexKey, edgeValue);
-
-		// the vertex value can be provided by an input file or a user-defined mapper
-		if (vertexReader != null) {
-			vertices = vertexReader.types(vertexKey, vertexValue);
-			return Graph.fromTupleDataSet(vertices, edges, executionContext);
-		}
-		else if (mapper != null) {
-			return Graph.fromTupleDataSet(edges, (MapFunction<K, VV>) mapper, executionContext);
-		}
-		else {
-			throw new RuntimeException("Vertex values have to be specified through a vertices input file"
-					+ "or a user-defined map function.");
-		}
-	}
-
-	/**
-	 * Creates a Graph from CSV input with edge values, but without vertex values.
-	 * @param vertexKey the type of the vertex IDs
-	 * @param edgeValue the type of the edge values
-	 * @return a Graph where the edges are read from an edges CSV file (with values).
-	 */
-	public <K, EV> Graph<K, NullValue, EV> edgeTypes(Class<K> vertexKey, Class<EV> edgeValue) {
-
-		if (edgeReader == null) {
-			throw new RuntimeException("The edges input file cannot be null!");
-		}
-
-		DataSet<Tuple3<K, K, EV>> edges = edgeReader.types(vertexKey, vertexKey, edgeValue);
-
-		return Graph.fromTupleDataSet(edges, executionContext);
-	}
-
-	/**
-	 * Creates a Graph from CSV input without vertex values or edge values.
-	 * @param vertexKey the type of the vertex IDs
-	 * @return a Graph where the vertex IDs are read from the edges input file.
-	 */
-	public <K> Graph<K, NullValue, NullValue> keyType(Class<K> vertexKey) {
-
-		if (edgeReader == null) {
-			throw new RuntimeException("The edges input file cannot be null!");
-		}
-
-		@SuppressWarnings("serial")
-		DataSet<Tuple3<K, K, NullValue>> edges = edgeReader.types(vertexKey, vertexKey)
-				.map(new MapFunction<Tuple2<K, K>, Tuple3<K, K, NullValue>>() {
-
-					public Tuple3<K, K, NullValue> map(Tuple2<K, K> edge) {
-						return new Tuple3<K, K, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
-					}
-				}).withForwardedFields("f0;f1");;
-
-		return Graph.fromTupleDataSet(edges, executionContext);
-	}
-
-	/**
-	 * Creates a Graph from CSV input without edge values.
-	 * The vertex values are specified through a vertices input file or a user-defined map function.
-	 * If no vertices input file is provided, the vertex IDs are automatically created from the edges
-	 * input file.
-	 * @param vertexKey the type of the vertex IDs
-	 * @param vertexValue the type of the vertex values
-	 * @return a Graph where the vertex IDs and vertex values.
-	 */
-	@SuppressWarnings({ "serial", "unchecked" })
-	public <K, VV> Graph<K, VV, NullValue> vertexTypes(Class<K> vertexKey, Class<VV> vertexValue) {
-		
-		DataSet<Tuple2<K, VV>> vertices = null;
-
-		if (edgeReader == null) {
-			throw new RuntimeException("The edges input file cannot be null!");
-		}
-
-		DataSet<Tuple3<K, K, NullValue>> edges = edgeReader.types(vertexKey, vertexKey)
-				.map(new MapFunction<Tuple2<K,K>, Tuple3<K, K, NullValue>>() {
-
-					public Tuple3<K, K, NullValue> map(Tuple2<K, K> input) {
-						return new Tuple3<K, K, NullValue>(input.f0, input.f1, NullValue.getInstance());
-					}
-				}).withForwardedFields("f0;f1");
-
-		// the vertex value can be provided by an input file or a user-defined mapper
-		if (vertexReader != null) {
-			vertices = vertexReader.types(vertexKey, vertexValue);
-			return Graph.fromTupleDataSet(vertices, edges, executionContext);
-		}
-		else if (mapper != null) {
-			return Graph.fromTupleDataSet(edges, (MapFunction<K, VV>) mapper, executionContext);
-		}
-		else {
-			throw new RuntimeException("Vertex values have to be specified through a vertices input file"
-					+ "or a user-defined map function.");
-		}
-	}
-
-	/**
-	 *Configures the Delimiter that separates rows for the CSV reader used to read the edges
-	 *	({@code '\n'}) is used by default.
-	 *
-	 *@param delimiter The delimiter that separates the rows.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader lineDelimiterEdges(String delimiter) {
-		edgeReader.lineDelimiter(delimiter);
-		return this;
-	}
-
-	/**
-	 *Configures the Delimiter that separates rows for the CSV reader used to read the vertices
-	 *	({@code '\n'}) is used by default.
-	 *
-	 *@param delimiter The delimiter that separates the rows.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader lineDelimiterVertices(String delimiter) {
-		if(this.vertexReader != null) {
-			this.vertexReader.lineDelimiter(delimiter);
-		}
-		return this;
-	}
-
-	/**
-	 *Configures the Delimiter that separates fields in a row for the CSV reader used to read the vertices
-	 * ({@code ','}) is used by default.
-	 *
-	 * @param delimiter The delimiter that separates the fields in a row.
-	 * @return The GraphCsv reader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader fieldDelimiterVertices(String delimiter) {
-		if(this.vertexReader != null) {
-			this.vertexReader.fieldDelimiter(delimiter);
-		}
-		return this;
-	}
-
-	/**
-	 *Configures the Delimiter that separates fields in a row for the CSV reader used to read the edges
-	 * ({@code ','}) is used by default.
-	 *
-	 * @param delimiter The delimiter that separates the fields in a row.
-	 * @return The GraphCsv reader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader fieldDelimiterEdges(String delimiter) {
-		this.edgeReader.fieldDelimiter(delimiter);
-		return this;
-	}
-
-	/**
-	 * Enables quoted String parsing for Edge Csv Reader. Field delimiters in quoted Strings are ignored.
-	 * A String is parsed as quoted if it starts and ends with a quoting character and as unquoted otherwise.
-	 * Leading or tailing whitespaces are not allowed.
-	 *
-	 * @param quoteCharacter The character which is used as quoting character.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader parseQuotedStringsEdges(char quoteCharacter) {
-		this.edgeReader.parseQuotedStrings(quoteCharacter);
-		return this;
-	}
-
-	/**
-	 * Enables quoted String parsing for Vertex Csv Reader. Field delimiters in quoted Strings are ignored.
-	 * A String is parsed as quoted if it starts and ends with a quoting character and as unquoted otherwise.
-	 * Leading or tailing whitespaces are not allowed.
-	 *
-	 * @param quoteCharacter The character which is used as quoting character.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader parseQuotedStringsVertices(char quoteCharacter) {
-		if(this.vertexReader != null) {
-			this.vertexReader.parseQuotedStrings(quoteCharacter);
-		}
-		return this;
-	}
-
-	/**
-	 * Configures the string that starts comments for the Vertex Csv Reader.
-	 * By default comments will be treated as invalid lines.
-	 * This function only recognizes comments which start at the beginning of the line!
-	 *
-	 * @param commentPrefix The string that starts the comments.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreCommentsVertices(String commentPrefix) {
-		if(this.vertexReader != null) {
-			this.vertexReader.ignoreComments(commentPrefix);
-		}
-		return this;
-	}
-
-	/**
-	 * Configures the string that starts comments for the Edge Csv Reader.
-	 * By default comments will be treated as invalid lines.
-	 * This function only recognizes comments which start at the beginning of the line!
-	 *
-	 * @param commentPrefix The string that starts the comments.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreCommentsEdges(String commentPrefix) {
-		this.edgeReader.ignoreComments(commentPrefix);
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
-	 * parser will look at the first {@code n} fields, where {@code n} is the length of the boolean
-	 * array. The parser will skip over all fields where the boolean value at the corresponding position
-	 * in the array is {@code false}. The result contains the fields where the corresponding position in
-	 * the boolean array is {@code true}.
-	 * The number of fields in the result is consequently equal to the number of times that {@code true}
-	 * occurs in the fields array.
-	 *
-	 * @param vertexFields The array of flags that describes which fields are to be included from the CSV file for vertices.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsVertices(boolean ... vertexFields) {
-		if(this.vertexReader != null) {
-			this.vertexReader.includeFields(vertexFields);
-		}
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
-	 * parser will look at the first {@code n} fields, where {@code n} is the length of the boolean
-	 * array. The parser will skip over all fields where the boolean value at the corresponding position
-	 * in the array is {@code false}. The result contains the fields where the corresponding position in
-	 * the boolean array is {@code true}.
-	 * The number of fields in the result is consequently equal to the number of times that {@code true}
-	 * occurs in the fields array.
-	 *
-	 * @param edgeFields The array of flags that describes which fields are to be included from the CSV file for edges.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsEdges(boolean ... edgeFields) {
-		this.edgeReader.includeFields(edgeFields);
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
-	 * positions in the string (read from position 0 to its length) define whether the field at
-	 * the corresponding position in the CSV schema should be included.
-	 * parser will look at the first {@code n} fields, where {@code n} is the length of the mask string
-	 * The parser will skip over all fields where the character at the corresponding position
-	 * in the string is {@code '0'}, {@code 'F'}, or {@code 'f'} (representing the value
-	 * {@code false}). The result contains the fields where the corresponding position in
-	 * the boolean array is {@code '1'}, {@code 'T'}, or {@code 't'} (representing the value {@code true}).
-	 *
-	 * @param mask The string mask defining which fields to include and which to skip.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsVertices(String mask) {
-		if(this.vertexReader != null) {
-			this.vertexReader.includeFields(mask);
-		}
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
-	 * positions in the string (read from position 0 to its length) define whether the field at
-	 * the corresponding position in the CSV schema should be included.
-	 * parser will look at the first {@code n} fields, where {@code n} is the length of the mask string
-	 * The parser will skip over all fields where the character at the corresponding position
-	 * in the string is {@code '0'}, {@code 'F'}, or {@code 'f'} (representing the value
-	 * {@code false}). The result contains the fields where the corresponding position in
-	 * the boolean array is {@code '1'}, {@code 'T'}, or {@code 't'} (representing the value {@code true}).
-	 *
-	 * @param mask The string mask defining which fields to include and which to skip.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsEdges(String mask) {
-		this.edgeReader.includeFields(mask);
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing vertices data should be included and which should be skipped. The
-	 * bits in the value (read from least significant to most significant) define whether the field at
-	 * the corresponding position in the CSV schema should be included.
-	 * parser will look at the first {@code n} fields, where {@code n} is the position of the most significant
-	 * non-zero bit.
-	 * The parser will skip over all fields where the character at the corresponding bit is zero, and
-	 * include the fields where the corresponding bit is one.
-	 * <p>
-	 * Examples:
-	 * <ul>
-	 *   <li>A mask of {@code 0x7} would include the first three fields.</li>
-	 *   <li>A mask of {@code 0x26} (binary {@code 100110} would skip the first fields, include fields
-	 *       two and three, skip fields four and five, and include field six.</li>
-	 * </ul>
-	 *
-	 * @param mask The bit mask defining which fields to include and which to skip.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsVertices(long mask) {
-		if(this.vertexReader != null) {
-			this.vertexReader.includeFields(mask);
-		}
-		return this;
-	}
-
-	/**
-	 * Configures which fields of the CSV file containing edges data should be included and which should be skipped. The
-	 * bits in the value (read from least significant to most significant) define whether the field at
-	 * the corresponding position in the CSV schema should be included.
-	 * parser will look at the first {@code n} fields, where {@code n} is the position of the most significant
-	 * non-zero bit.
-	 * The parser will skip over all fields where the character at the corresponding bit is zero, and
-	 * include the fields where the corresponding bit is one.
-	 * <p>
-	 * Examples:
-	 * <ul>
-	 *   <li>A mask of {@code 0x7} would include the first three fields.</li>
-	 *   <li>A mask of {@code 0x26} (binary {@code 100110} would skip the first fields, include fields
-	 *       two and three, skip fields four and five, and include field six.</li>
-	 * </ul>
-	 *
-	 * @param mask The bit mask defining which fields to include and which to skip.
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader includeFieldsEdges(long mask) {
-		this.edgeReader.includeFields(mask);
-		return this;
-	}
-
-	/**
-	 * Sets the CSV reader for the Edges file to ignore the first line. This is useful for files that contain a header line.
-	 *
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreFirstLineEdges() {
-		this.edgeReader.ignoreFirstLine();
-		return this;
-	}
-
-	/**
-	 * Sets the CSV reader for the Vertices file to ignore the first line. This is useful for files that contain a header line.
-	 *
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreFirstLineVertices() {
-		if(this.vertexReader != null) {
-			this.vertexReader.ignoreFirstLine();
-		}
-		return this;
-	}
-
-	/**
-	 * Sets the CSV reader for the Edges file  to ignore any invalid lines.
-	 * This is useful for files that contain an empty line at the end, multiple header lines or comments. This would throw an exception otherwise.
-	 *
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreInvalidLinesEdges() {
-		this.edgeReader.ignoreInvalidLines();
-		return this;
-	}
-
-	/**
-	 * Sets the CSV reader Vertices file  to ignore any invalid lines.
-	 * This is useful for files that contain an empty line at the end, multiple header lines or comments. This would throw an exception otherwise.
-	 *
-	 * @return The GraphCSVReader instance itself, to allow for fluent function chaining.
-	 */
-	public GraphCsvReader ignoreInvalidLinesVertices() {
-		if(this.vertexReader != null) {
-			this.vertexReader.ignoreInvalidLines();
-		}
-		return this;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
deleted file mode 100644
index 3215194..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/IterationConfiguration.java
+++ /dev/null
@@ -1,155 +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;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import com.google.common.base.Preconditions;
-
-/**
- * This is used as a base class for vertex-centric iteration or gather-sum-apply iteration configuration.
- */
-public abstract class IterationConfiguration {
-
-	/** the iteration name **/
-	private String name;
-
-	/** the iteration parallelism **/
-	private int parallelism = -1;
-
-	/** the iteration aggregators **/
-	private Map<String, Aggregator<?>> aggregators = new HashMap<String, Aggregator<?>>();
-
-	/** flag that defines whether the solution set is kept in managed memory **/
-	private boolean unmanagedSolutionSet = false;
-
-	/** flag that defines whether the number of vertices option is set **/
-	private boolean optNumVertices = false;
-	
-	public IterationConfiguration() {}
-
-	/**
-	 * Sets the name for the iteration. The name is displayed in logs and messages.
-	 * 
-	 * @param name The name for the iteration.
-	 */
-	public void setName(String name) {
-		this.name = name;
-	}
-
-	/**
-	 * Gets the name of the iteration.
-	 * @param defaultName 
-	 * 
-	 * @return The name of the iteration.
-	 */
-	public String getName(String defaultName) {
-		if (name != null) {
-			return name;			
-		}
-		else {
-			return defaultName;
-		}
-	}
-
-	/**
-	 * Sets the parallelism for the iteration.
-	 * 
-	 * @param parallelism The parallelism.
-	 */
-	public void setParallelism(int parallelism) {
-		Preconditions.checkArgument(parallelism > 0 || parallelism == -1, "The parallelism must be positive, or -1 (use default).");
-		this.parallelism = parallelism;
-	}
-	
-	/**
-	 * Gets the iteration's parallelism.
-	 * 
-	 * @return The iterations parallelism, or -1, if not set.
-	 */
-	public int getParallelism() {
-		return parallelism;
-	}
-
-	/**
-	 * Defines whether the solution set is kept in managed memory (Flink's internal way of keeping object
-	 * in serialized form) or as a simple object map.
-	 * By default, the solution set runs in managed memory.
-	 * 
-	 * @param unmanaged True, to keep the solution set in unmanaged memory, false otherwise.
-	 */
-	public void setSolutionSetUnmanagedMemory(boolean unmanaged) {
-		this.unmanagedSolutionSet = unmanaged;
-	}
-	
-	/**
-	 * Gets whether the solution set is kept in managed memory (Flink's internal way of keeping object
-	 * in serialized form) or as a simple object map.
-	 * By default, the solution set runs in managed memory.
-	 * 
-	 * @return True, if the solution set is in unmanaged memory, false otherwise.
-	 */
-	public boolean isSolutionSetUnmanagedMemory() {
-		return this.unmanagedSolutionSet;
-	}
-
-	/**
-	 * Gets whether the number of vertices option is set.
-	 * By default, the number of vertices option is not set.
-	 *
-	 * @return True, if the number of vertices option is set, false otherwise.
-	 */
-	public boolean isOptNumVertices() {
-		return optNumVertices;
-	}
-
-	/**
-	 * Sets the number of vertices option.
-	 * By default, the number of vertices option is not set.
-	 *
-	 * @param optNumVertices True, to set this option, false otherwise.
-	 */
-	public void setOptNumVertices(boolean optNumVertices) {
-		this.optNumVertices = optNumVertices;
-	}
-
-	/**
-	 * Registers a new aggregator. Aggregators registered here are available during the execution of the vertex updates
-	 * via {@link org.apache.flink.graph.spargel.VertexUpdateFunction#getIterationAggregator(String)} and
-	 * {@link org.apache.flink.graph.spargel.VertexUpdateFunction#getPreviousIterationAggregate(String)}.
-	 * 
-	 * @param name The name of the aggregator, used to retrieve it and its aggregates during execution. 
-	 * @param aggregator The aggregator.
-	 */
-	public void registerAggregator(String name, Aggregator<?> aggregator) {
-		this.aggregators.put(name, aggregator);
-	}
-
-	/**
-	 * Gets the set of aggregators that are registered for this vertex-centric iteration.
-	 *
-	 * @return a Map of the registered aggregators, where the key is the aggregator name
-	 * and the value is the Aggregator object
-	 */
-	public Map<String, Aggregator<?>> getAggregators() {
-		return this.aggregators;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
deleted file mode 100644
index a21b23d..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunction.java
+++ /dev/null
@@ -1,40 +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;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.util.Collector;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#groupReduceOnNeighbors(NeighborsFunction, EdgeDirection)}
- * method.
- *
- * @param <K> the vertex key type
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- * @param <O> the type of the return value
- */
-public interface NeighborsFunction<K, VV, EV, O> extends Function, Serializable {
-
-	void iterateNeighbors(Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
deleted file mode 100644
index fdf54fa..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/NeighborsFunctionWithVertexValue.java
+++ /dev/null
@@ -1,40 +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;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#groupReduceOnNeighbors(NeighborsFunctionWithVertexValue, EdgeDirection)}
- * method.
- *
- * @param <K> the vertex key type
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- * @param <O> the type of the return value
- */
-public interface NeighborsFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
-
-	void iterateNeighbors(Vertex<K, VV> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
deleted file mode 100644
index 84eec51..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceEdgesFunction.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.graph;
-
-import org.apache.flink.api.common.functions.Function;
-
-import java.io.Serializable;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#reduceOnEdges(org.apache.flink.graph.ReduceEdgesFunction, EdgeDirection)} method.
- *
- * @param <EV> the edge value type
- */
-public interface ReduceEdgesFunction<EV> extends Function, Serializable {
-
-	EV reduceEdges(EV firstEdgeValue, EV secondEdgeValue);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
deleted file mode 100644
index fc5295d..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/ReduceNeighborsFunction.java
+++ /dev/null
@@ -1,35 +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;
-
-import org.apache.flink.api.common.functions.Function;
-
-import java.io.Serializable;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#reduceOnNeighbors(ReduceNeighborsFunction, EdgeDirection)}
- * method.
- *
- * @param <VV> the vertex value type
- */
-public interface ReduceNeighborsFunction <VV> extends Function, Serializable {
-
-	VV reduceNeighbors(VV firstNeighborValue, VV secondNeighborValue);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
deleted file mode 100644
index dee3480..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Triplet.java
+++ /dev/null
@@ -1,77 +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;
-
-import org.apache.flink.api.java.tuple.Tuple5;
-
-/**
- * A Triplet stores and retrieves the edges along with their corresponding source and target vertices.
- * Triplets can be obtained from the input graph via the {@link org.apache.flink.graph.Graph#getTriplets()} method.
- *
- * @param <K> the vertex key type
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- */
-public class Triplet <K, VV, EV> extends Tuple5<K, K, VV, VV, EV> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Triplet() {}
-
-	/**
-	 * Constructs a Triplet from a given source vertex, target vertex and edge
-	 *
-	 * @param srcVertex
-	 * @param trgVertex
-	 * @param edge
-	 */
-	public Triplet(Vertex<K, VV> srcVertex, Vertex<K, VV> trgVertex, Edge<K, EV> edge) {
-		this.f0 = srcVertex.f0;
-		this.f2 = srcVertex.f1;
-		this.f1 = trgVertex.f0;
-		this.f3 = trgVertex.f1;
-		this.f4 = edge.f2;
-	}
-
-	/**
-	 * Constructs a Triplet from its src vertex id, src target id, src vertex value,
-	 * src target value and edge value respectively.
-	 *
-	 * @param srcId
-	 * @param trgId
-	 * @param srcVal
-	 * @param trgVal
-	 * @param edgeVal
-	 */
-	public Triplet(K srcId, K trgId, VV srcVal, VV trgVal, EV edgeVal) {
-		super(srcId, trgId, srcVal, trgVal, edgeVal);
-	}
-
-	public Vertex<K, VV> getSrcVertex() {
-		return new Vertex<K, VV>(this.f0, this.f2);
-	}
-
-	public Vertex<K, VV> getTrgVertex() {
-		return new Vertex<K, VV>(this.f1, this.f3);
-	}
-
-	public Edge<K, EV> getEdge() {
-		return new Edge<K, EV>(this.f0, this.f1, this.f4);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java
deleted file mode 100644
index c5eb973..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Vertex.java
+++ /dev/null
@@ -1,56 +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;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
-/**
- * Represents the graph's nodes. It carries an ID and a value.
- * For vertices with no value, use {@link org.apache.flink.types.NullValue} as the value type.
- *
- * @param <K>
- * @param <V>
- */
-public class Vertex<K, V> extends Tuple2<K, V> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Vertex(){}
-
-	public Vertex(K k, V val) {
-		this.f0 = k;
-		this.f1 = val;
-	}
-
-	public K getId() {
-		return this.f0;
-	}
-
-	public V getValue() {
-		return this.f1;
-	}
-
-	public void setId(K id) {
-		this.f0 = id;
-	}
-
-	public void setValue(V val) {
-		this.f1 = val;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ConnectedComponents.java
deleted file mode 100644
index cd52e04..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/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.example;
-
-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.example.utils.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 org.apache.flink.graph.library.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 org.apache.flink.graph.example.utils.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, 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<Long, NullValue>(value.f0, value.f1, NullValue.getInstance());
-						}
-					});
-		} else {
-			return ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env);
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
deleted file mode 100644
index b7e3385..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
+++ /dev/null
@@ -1,215 +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.example;
-
-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.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Triplet;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.EuclideanGraphData;
-
-import java.io.Serializable;
-
-/**
- * This example shows how to use Gelly's {@link Graph#getTriplets()} and
- * {@link Graph#joinWithEdges(DataSet, MapFunction)} methods.
- * 
- * Given a directed, unweighted graph, with vertex values representing points in a plan,
- * return a weighted graph where the edge weights are equal to the Euclidean distance between the
- * src and the trg vertex values.
- *
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <ul>
- * 	<li> Vertices are represented by their vertexIds and vertex values and are separated by newlines,
- * 	the value being formed of two doubles separated by a comma.
- * 	For example: <code>1,1.0,1.0\n2,2.0,2.0\n3,3.0,3.0\n</code> defines a data set of three vertices
- * 	<li> Edges are represented by pairs of srcVertexId, trgVertexId separated by commas.
- * 	Edges themselves are separated by newlines.
- * 	For example: <code>1,2\n1,3\n</code> defines two edges 1-2 and 1-3.
- * </ul>
- * </p>
- *
- * Usage <code>EuclideanGraphWeighing &lt;vertex path&gt; &lt;edge path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.graph.example.utils.EuclideanGraphData}
- */
-@SuppressWarnings("serial")
-public class EuclideanGraphWeighing implements ProgramDescription {
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Vertex<Long, Point>> vertices = getVerticesDataSet(env);
-
-		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
-
-		Graph<Long, Point, Double> graph = Graph.fromDataSet(vertices, edges, env);
-
-		// the edge value will be the Euclidean distance between its src and trg vertex
-		DataSet<Tuple3<Long, Long, Double>> edgesWithEuclideanWeight = graph.getTriplets()
-				.map(new MapFunction<Triplet<Long, Point, Double>, Tuple3<Long, Long, Double>>() {
-
-					@Override
-					public Tuple3<Long, Long, Double> map(Triplet<Long, Point, Double> triplet)
-							throws Exception {
-
-						Vertex<Long, Point> srcVertex = triplet.getSrcVertex();
-						Vertex<Long, Point> trgVertex = triplet.getTrgVertex();
-
-						return new Tuple3<Long, Long, Double>(srcVertex.getId(), trgVertex.getId(),
-								srcVertex.getValue().euclideanDistance(trgVertex.getValue()));
-					}
-				});
-
-		Graph<Long, Point, Double> resultedGraph = graph.joinWithEdges(edgesWithEuclideanWeight,
-				new MapFunction<Tuple2<Double, Double>, Double>() {
-
-					@Override
-					public Double map(Tuple2<Double, Double> distance) throws Exception {
-						return distance.f1;
-					}
-				});
-
-		// retrieve the edges from the final result
-		DataSet<Edge<Long, Double>> result = resultedGraph.getEdges();
-
-		// emit result
-		if (fileOutput) {
-			result.writeAsCsv(outputPath, "\n", ",");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute("Euclidean Graph Weighing Example");
-		} else {
-			result.print();
-		}
-
-	}
-
-	@Override
-	public String getDescription() {
-		return "Weighing a graph by computing the Euclidean distance " +
-				"between its vertices";
-	}
-
-	// *************************************************************************
-	//     DATA TYPES
-	// *************************************************************************
-
-	/**
-	 * A simple two-dimensional point.
-	 */
-	public static class Point implements Serializable {
-
-		public double x, y;
-
-		public Point() {}
-
-		public Point(double x, double y) {
-			this.x = x;
-			this.y = y;
-		}
-
-		public double euclideanDistance(Point other) {
-			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
-		}
-
-		@Override
-		public String toString() {
-			return x + " " + y;
-		}
-	}
-
-	// ******************************************************************************************************************
-	// UTIL METHODS
-	// ******************************************************************************************************************
-
-	private static boolean fileOutput = false;
-
-	private static String verticesInputPath = null;
-
-	private static String edgesInputPath = null;
-
-	private static String outputPath = null;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			if (args.length == 3) {
-				fileOutput = true;
-				verticesInputPath = args[0];
-				edgesInputPath = args[1];
-				outputPath = args[2];
-			} else {
-				System.out.println("Executing Euclidean Graph Weighing example with default parameters and built-in default data.");
-				System.out.println("Provide parameters to read input data from files.");
-				System.out.println("See the documentation for the correct format of input files.");
-				System.err.println("Usage: EuclideanGraphWeighing <input vertices path> <input edges path>" +
-						" <output path>");
-				return false;
-			}
-		}
-		return true;
-	}
-
-	private static DataSet<Vertex<Long, Point>> getVerticesDataSet(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(verticesInputPath)
-					.lineDelimiter("\n")
-					.types(Long.class, Double.class, Double.class)
-					.map(new MapFunction<Tuple3<Long, Double, Double>, Vertex<Long, Point>>() {
-
-						@Override
-						public Vertex<Long, Point> map(Tuple3<Long, Double, Double> value) throws Exception {
-							return new Vertex<Long, Point>(value.f0, new Point(value.f1, value.f2));
-						}
-					});
-		} else {
-			return EuclideanGraphData.getDefaultVertexDataSet(env);
-		}
-	}
-
-	private static DataSet<Edge<Long, Double>> getEdgesDataSet(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(edgesInputPath)
-					.lineDelimiter("\n")
-					.types(Long.class, Long.class)
-					.map(new MapFunction<Tuple2<Long, Long>, Edge<Long, Double>>() {
-
-						@Override
-						public Edge<Long, Double> map(Tuple2<Long, Long> tuple2) throws Exception {
-							return new Edge<Long, Double>(tuple2.f0, tuple2.f1, 0.0);
-						}
-					});
-		} else {
-			return EuclideanGraphData.getDefaultEdgeDataSet(env);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
deleted file mode 100755
index 635a099..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSASingleSourceShortestPaths.java
+++ /dev/null
@@ -1,191 +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.example;
-
-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.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
-import org.apache.flink.graph.gsa.ApplyFunction;
-import org.apache.flink.graph.gsa.GatherFunction;
-import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.graph.gsa.Neighbor;
-import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
-
-/**
- * This example shows how to use Gelly's Gather-Sum-Apply iterations.
- * 
- * It is an implementation of the Single-Source-Shortest-Paths algorithm.
- * For a vertex-centric implementation of the same algorithm, please refer to {@link SingleSourceShortestPaths}. 
- *
- * The input file is a plain text file and must be formatted as follows:
- * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
- * separated by tabs. Edges themselves are separated by newlines.
- * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
- * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
- *
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData}
- */
-public class GSASingleSourceShortestPaths implements ProgramDescription {
-
-	// --------------------------------------------------------------------------------------------
-	//  Program
-	// --------------------------------------------------------------------------------------------
-
-	public static void main(String[] args) throws Exception {
-
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Edge<Long, Double>> edges = getEdgeDataSet(env);
-
-		Graph<Long, Double, Double> graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env);
-
-		// Execute the GSA iteration
-		Graph<Long, Double, Double> result = graph.runGatherSumApplyIteration(
-				new CalculateDistances(), new ChooseMinDistance(), new UpdateDistance(), maxIterations);
-
-		// Extract the vertices as the result
-		DataSet<Vertex<Long, Double>> singleSourceShortestPaths = result.getVertices();
-
-		// emit result
-		if(fileOutput) {
-			singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute("GSA Single Source Shortest Paths");
-		} else {
-			singleSourceShortestPaths.print();
-		}
-
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Single Source Shortest Path UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	private static final class InitVertices implements MapFunction<Long, Double>{
-
-		private long srcId;
-
-		public InitVertices(long srcId) {
-			this.srcId = srcId;
-		}
-
-		public Double map(Long id) {
-			if (id.equals(srcId)) {
-				return 0.0;
-			}
-			else {
-				return Double.POSITIVE_INFINITY;
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class CalculateDistances extends GatherFunction<Double, Double, Double> {
-
-		public Double gather(Neighbor<Double, Double> neighbor) {
-			return neighbor.getNeighborValue() + neighbor.getEdgeValue();
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class ChooseMinDistance extends SumFunction<Double, Double, Double> {
-
-		public Double sum(Double newValue, Double currentValue) {
-			return Math.min(newValue, currentValue);
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class UpdateDistance extends ApplyFunction<Long, Double, Double> {
-
-		public void apply(Double newDistance, Double oldDistance) {
-			if (newDistance < oldDistance) {
-				setResult(newDistance);
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Util methods
-	// --------------------------------------------------------------------------------------------
-
-	private static boolean fileOutput = false;
-
-	private static Long srcVertexId = 1l;
-
-	private static String edgesInputPath = null;
-
-	private static String outputPath = null;
-
-	private static int maxIterations = 5;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			if(args.length != 4) {
-				System.err.println("Usage: GSASingleSourceShortestPaths <source vertex id>" +
-						" <input edges path> <output path> <num iterations>");
-				return false;
-			}
-
-			fileOutput = true;
-			srcVertexId = Long.parseLong(args[0]);
-			edgesInputPath = args[1];
-			outputPath = args[2];
-			maxIterations = Integer.parseInt(args[3]);
-		} else {
-				System.out.println("Executing GSASingle Source Shortest Paths example "
-						+ "with default parameters and built-in default data.");
-				System.out.println("  Provide parameters to read input data from files.");
-				System.out.println("  See the documentation for the correct format of input files.");
-				System.out.println("Usage: GSASingleSourceShortestPaths <source vertex id>" +
-						" <input edges path> <output path> <num iterations>");
-		}
-		return true;
-	}
-
-	private static DataSet<Edge<Long, Double>> getEdgeDataSet(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(edgesInputPath)
-					.fieldDelimiter("\t")
-					.lineDelimiter("\n")
-					.types(Long.class, Long.class, Double.class)
-					.map(new Tuple3ToEdgeMap<Long, Double>());
-		} else {
-			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
-		}
-	}
-
-	@Override
-	public String getDescription() {
-		return "GSA Single Source Shortest Paths";
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
deleted file mode 100644
index b808e76..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
+++ /dev/null
@@ -1,170 +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.example;
-
-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.aggregation.Aggregations;
-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.example.utils.ExampleUtils;
-import org.apache.flink.types.NullValue;
-
-/**
- * This example illustrates how to use Gelly metrics methods and get simple statistics
- * from the input graph.  
- * 
- * The program creates a random graph and computes and prints
- * the following metrics:
- * - number of vertices
- * - number of edges
- * - average node degree
- * - the vertex ids with the max/min in- and out-degrees
- *
- * The input file is expected to contain one edge per line,
- * with long IDs and no values, in the following format:
- * "<sourceVertexID>\t<targetVertexID>".
- * If no arguments are provided, the example runs with a random graph of 100 vertices.
- *
- */
-public class GraphMetrics implements ProgramDescription {
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		/** create the graph **/
-		Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(getEdgesDataSet(env), env);
-		
-		/** get the number of vertices **/
-		long numVertices = graph.numberOfVertices();
-		
-		/** get the number of edges **/
-		long numEdges = graph.numberOfEdges();
-		
-		/** compute the average node degree **/
-		DataSet<Tuple2<Long, Long>> verticesWithDegrees = graph.getDegrees();
-
-		DataSet<Double> avgNodeDegree = verticesWithDegrees
-				.aggregate(Aggregations.SUM, 1).map(new AvgNodeDegreeMapper(numVertices));
-		
-		/** find the vertex with the maximum in-degree **/
-		DataSet<Long> maxInDegreeVertex = graph.inDegrees().maxBy(1).map(new ProjectVertexId());
-
-		/** find the vertex with the minimum in-degree **/
-		DataSet<Long> minInDegreeVertex = graph.inDegrees().minBy(1).map(new ProjectVertexId());
-
-		/** find the vertex with the maximum out-degree **/
-		DataSet<Long> maxOutDegreeVertex = graph.outDegrees().maxBy(1).map(new ProjectVertexId());
-
-		/** find the vertex with the minimum out-degree **/
-		DataSet<Long> minOutDegreeVertex = graph.outDegrees().minBy(1).map(new ProjectVertexId());
-		
-		/** print the results **/
-		ExampleUtils.printResult(env.fromElements(numVertices), "Total number of vertices");
-		ExampleUtils.printResult(env.fromElements(numEdges), "Total number of edges");
-		ExampleUtils.printResult(avgNodeDegree, "Average node degree");
-		ExampleUtils.printResult(maxInDegreeVertex, "Vertex with Max in-degree");
-		ExampleUtils.printResult(minInDegreeVertex, "Vertex with Min in-degree");
-		ExampleUtils.printResult(maxOutDegreeVertex, "Vertex with Max out-degree");
-		ExampleUtils.printResult(minOutDegreeVertex, "Vertex with Min out-degree");
-
-		env.execute();
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AvgNodeDegreeMapper implements MapFunction<Tuple2<Long, Long>, Double> {
-
-		private long numberOfVertices;
-
-		public AvgNodeDegreeMapper(long numberOfVertices) {
-			this.numberOfVertices = numberOfVertices;
-		}
-
-		public Double map(Tuple2<Long, Long> sumTuple) {
-			return (double) (sumTuple.f1 / numberOfVertices) ;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ProjectVertexId implements MapFunction<Tuple2<Long,Long>, Long> {
-		public Long map(Tuple2<Long, Long> value) { return value.f0; }
-	}
-
-	@Override
-	public String getDescription() {
-		return "Graph Metrics Example";
-	}
-
-	// ******************************************************************************************************************
-	// UTIL METHODS
-	// ******************************************************************************************************************
-
-	private static boolean fileOutput = false;
-
-	private static String edgesInputPath = null;
-
-	static final int NUM_VERTICES = 100;
-
-	static final long SEED = 9876;
-
-	private static boolean parseParameters(String[] args) {
-
-		if(args.length > 0) {
-			if(args.length != 1) {
-				System.err.println("Usage: GraphMetrics <input edges>");
-				return false;
-			}
-
-			fileOutput = true;
-			edgesInputPath = args[0];
-		} else {
-			System.out.println("Executing Graph Metrics example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("Usage: GraphMetrics <input edges>");
-		}
-		return true;
-	}
-
-	@SuppressWarnings("serial")
-	private static DataSet<Edge<Long, NullValue>> getEdgesDataSet(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(edgesInputPath)
-					.lineDelimiter("\n").fieldDelimiter("\t")
-					.types(Long.class, Long.class).map(
-							new MapFunction<Tuple2<Long, Long>, Edge<Long, NullValue>>() {
-
-								public Edge<Long, NullValue> map(Tuple2<Long, Long> value) {
-									return new Edge<Long, NullValue>(value.f0, value.f1, 
-											NullValue.getInstance());
-								}
-					});
-		} else {
-			return ExampleUtils.getRandomEdges(env, NUM_VERTICES);
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
deleted file mode 100644
index c03937d..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
+++ /dev/null
@@ -1,268 +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.example;
-
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.IncrementalSSSPData;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexCentricConfiguration;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-
-/**
- * This example illustrates how to 
- * <ul>
- *  <li> create a Graph directly from CSV files
- *  <li> use the vertex-centric iteration's messaging direction configuration option
- * </ul>
- * 
- * Incremental Single Sink Shortest Paths Example. Shortest Paths are incrementally updated
- * upon edge removal.
- *
- * The program takes as input the resulted graph after a SSSP computation,
- * an edge to be removed and the initial graph(i.e. before SSSP was computed).
- * In the following description, SP-graph is used as an abbreviation for
- * the graph resulted from the SSSP computation. We denote the edges that belong to this
- * graph by SP-edges.
- *
- * - If the removed edge does not belong to the SP-graph, no computation is necessary.
- * The edge is simply removed from the graph.
- * - If the removed edge is an SP-edge, then all nodes, whose shortest path contains the removed edge,
- * potentially require re-computation.
- * When the edge <u, v> is removed, v checks if it has another out-going SP-edge.
- * If yes, no further computation is required.
- * If v has no other out-going SP-edge, it invalidates its current value, by setting it to INF.
- * Then, it informs all its SP-in-neighbors by sending them an INVALIDATE message.
- * When a vertex u receives an INVALIDATE message from v, it checks whether it has another out-going SP-edge.
- * If not, it invalidates its current value and propagates the INVALIDATE message.
- * The propagation stops when a vertex with an alternative shortest path is reached
- * or when we reach a vertex with no SP-in-neighbors.
- *
- * Usage <code>IncrementalSSSP &lt;vertex path&gt; &lt;edge path&gt; &lt;edges in SSSP&gt;
- * &lt;src id edge to be removed&gt; &lt;trg id edge to be removed&gt; &lt;val edge to be removed&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 org.apache.flink.graph.example.utils.IncrementalSSSPData}
- */
-@SuppressWarnings("serial")
-public class IncrementalSSSP implements ProgramDescription {
-
-	public static void main(String [] args) throws Exception {
-
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Edge<Long, Double> edgeToBeRemoved = getEdgeToBeRemoved();
-
-		Graph<Long, Double, Double> graph = IncrementalSSSP.getGraph(env);
-
-		// Assumption: all minimum weight paths are kept
-		Graph<Long, Double, Double> ssspGraph = IncrementalSSSP.getSSSPGraph(env);
-
-		// remove the edge
-		graph.removeEdge(edgeToBeRemoved);
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		if(isInSSSP(edgeToBeRemoved, ssspGraph.getEdges())) {
-
-			parameters.setDirection(EdgeDirection.IN);
-			parameters.setOptDegrees(true);
-
-			// run the vertex centric iteration to propagate info
-			Graph<Long, Double, Double> result = ssspGraph.runVertexCentricIteration(new VertexDistanceUpdater(),
-					new InvalidateMessenger(edgeToBeRemoved), maxIterations, parameters);
-
-			DataSet<Vertex<Long, Double>> resultedVertices = result.getVertices();
-
-			// Emit results
-			if(fileOutput) {
-				resultedVertices.writeAsCsv(outputPath, "\n", ",");
-				env.execute("Incremental SSSP Example");
-			} else {
-				resultedVertices.print();
-			}
-		} else {
-			// print the vertices
-			if(fileOutput) {
-				graph.getVertices().writeAsCsv(outputPath, "\n", ",");
-				env.execute("Incremental SSSP Example");
-			} else {
-				graph.getVertices().print();
-			}
-		}
-	}
-
-	@Override
-	public String getDescription() {
-		return "Incremental Single Sink Shortest Paths Example";
-	}
-
-	// ******************************************************************************************************************
-	// IncrementalSSSP METHODS
-	// ******************************************************************************************************************
-
-	/**
-	 * Function that verifies whether the edge to be removed is part of the SSSP or not.
-	 * If it is, the src vertex will be invalidated.
-	 *
-	 * @param edgeToBeRemoved
-	 * @param edgesInSSSP
-	 * @return
-	 */
-	public static boolean isInSSSP(final Edge<Long, Double> edgeToBeRemoved, DataSet<Edge<Long, Double>> edgesInSSSP) throws Exception {
-
-		return edgesInSSSP.filter(new FilterFunction<Edge<Long, Double>>() {
-			@Override
-			public boolean filter(Edge<Long, Double> edge) throws Exception {
-				return edge.equals(edgeToBeRemoved);
-			}
-		}).count() > 0;
-	}
-
-	public static final class VertexDistanceUpdater extends VertexUpdateFunction<Long, Double, Double> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Double> vertex, MessageIterator<Double> inMessages) throws Exception {
-			if (inMessages.hasNext()) {
-				Long outDegree = getOutDegree() - 1;
-				// check if the vertex has another SP-Edge
-				if (outDegree > 0) {
-					// there is another shortest path from the source to this vertex
-				} else {
-					// set own value to infinity
-					setNewVertexValue(Double.MAX_VALUE);
-				}
-			}
-		}
-	}
-
-	public static final class InvalidateMessenger extends MessagingFunction<Long, Double, Double, Double> {
-
-		private Edge<Long, Double> edgeToBeRemoved;
-
-		public InvalidateMessenger(Edge<Long, Double> edgeToBeRemoved) {
-			this.edgeToBeRemoved = edgeToBeRemoved;
-		}
-
-		@Override
-		public void sendMessages(Vertex<Long, Double> vertex) throws Exception {
-
-
-			if(getSuperstepNumber() == 1) {
-				if(vertex.getId().equals(edgeToBeRemoved.getSource())) {
-					// activate the edge target
-					sendMessageTo(edgeToBeRemoved.getSource(), Double.MAX_VALUE);
-				}
-			}
-
-			if(getSuperstepNumber() > 1) {
-				// invalidate all edges
-				for(Edge<Long, Double> edge : getEdges()) {
-					sendMessageTo(edge.getSource(), Double.MAX_VALUE);
-				}
-			}
-		}
-	}
-
-	// ******************************************************************************************************************
-	// UTIL METHODS
-	// ******************************************************************************************************************
-
-	private static boolean fileOutput = false;
-
-	private static String verticesInputPath = null;
-
-	private static String edgesInputPath = null;
-
-	private static String edgesInSSSPInputPath = null;
-
-	private static Long srcEdgeToBeRemoved = null;
-
-	private static Long trgEdgeToBeRemoved = null;
-
-	private static Double valEdgeToBeRemoved = null;
-
-	private static String outputPath = null;
-
-	private static int maxIterations = 5;
-
-	private static boolean parseParameters(String[] args) {
-		if (args.length > 0) {
-			if (args.length == 8) {
-				fileOutput = true;
-				verticesInputPath = args[0];
-				edgesInputPath = args[1];
-				edgesInSSSPInputPath = args[2];
-				srcEdgeToBeRemoved = Long.parseLong(args[3]);
-				trgEdgeToBeRemoved = Long.parseLong(args[4]);
-				valEdgeToBeRemoved = Double.parseDouble(args[5]);
-				outputPath = args[6];
-				maxIterations = Integer.parseInt(args[7]);
-			} else {
-				System.out.println("Executing IncrementalSSSP example with default parameters and built-in default data.");
-				System.out.println("Provide parameters to read input data from files.");
-				System.out.println("See the documentation for the correct format of input files.");
-				System.out.println("Usage: IncrementalSSSP <vertex path> <edge path> <edges in SSSP> " +
-						"<src id edge to be removed> <trg id edge to be removed> <val edge to be removed> " +
-						"<output path> <max iterations>");
-
-				return false;
-			}
-		}
-		return true;
-	}
-
-	private static Graph<Long, Double, Double> getGraph(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return Graph.fromCsvReader(verticesInputPath, edgesInputPath, env).lineDelimiterEdges("\n")
-					.types(Long.class, Double.class, Double.class);
-		} else {
-			return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgeDataSet(env), env);
-		}
-	}
-
-	private static Graph<Long, Double, Double> getSSSPGraph(ExecutionEnvironment env) {
-		if(fileOutput) {
-			return Graph.fromCsvReader(verticesInputPath, edgesInSSSPInputPath, env).lineDelimiterEdges("\n")
-					.types(Long.class, Double.class, Double.class);
-		} else {
-			return Graph.fromDataSet(IncrementalSSSPData.getDefaultVertexDataSet(env), IncrementalSSSPData.getDefaultEdgesInSSSP(env), env);
-		}
-	}
-
-	private static Edge<Long, Double> getEdgeToBeRemoved() {
-		if (fileOutput) {
-			return new Edge<Long, Double>(srcEdgeToBeRemoved, trgEdgeToBeRemoved, valEdgeToBeRemoved);
-		} else {
-			return IncrementalSSSPData.getDefaultEdgeToBeRemoved();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
deleted file mode 100644
index 0f84dbb..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/JaccardSimilarityMeasure.java
+++ /dev/null
@@ -1,214 +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.example;
-
-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.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.ReduceNeighborsFunction;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.Triplet;
-import org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData;
-
-import java.util.HashSet;
-
-/**
- * This example shows how to use
- * <ul>
- *  <li> neighborhood methods
- *  <li> join with vertices
- *  <li> triplets
- * </ul>
- * 
- * Given a directed, unweighted graph, return a weighted graph where the edge values are equal
- * to the Jaccard similarity coefficient - the number of common neighbors divided by the the size
- * of the union of neighbor sets - for the src and target vertices.
- *
- * <p>
- * Input files are plain text files and must be formatted as follows:
- * <br>
- * 	Edges are represented by pairs of srcVertexId, trgVertexId separated by tabs.
- * 	Edges themselves are separated by newlines.
- * 	For example: <code>1	2\n1	3\n</code> defines two edges 1-2 and 1-3.
- * </p>
- *
- * Usage <code> JaccardSimilarityMeasure &lt;edge path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData}
- */
-@SuppressWarnings("serial")
-public class JaccardSimilarityMeasure implements ProgramDescription {
-
-	public static void main(String [] args) throws Exception {
-
-		if(!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
-
-		Graph<Long, HashSet<Long>, Double> graph = Graph.fromDataSet(edges,
-				new MapFunction<Long, HashSet<Long>>() {
-
-					@Override
-					public HashSet<Long> map(Long id) throws Exception {
-						HashSet<Long> neighbors = new HashSet<Long>();
-						neighbors.add(id);
-
-						return new HashSet<Long>(neighbors);
-					}
-				}, env);
-
-		// create the set of neighbors
-		DataSet<Tuple2<Long, HashSet<Long>>> computedNeighbors =
-				graph.reduceOnNeighbors(new GatherNeighbors(), EdgeDirection.ALL);
-
-		// join with the vertices to update the node values
-		Graph<Long, HashSet<Long>, Double> graphWithVertexValues =
-				graph.joinWithVertices(computedNeighbors, new MapFunction<Tuple2<HashSet<Long>, HashSet<Long>>,
-						HashSet<Long>>() {
-
-					@Override
-					public HashSet<Long> map(Tuple2<HashSet<Long>, HashSet<Long>> tuple2) throws Exception {
-						return tuple2.f1;
-					}
-				});
-
-		// compare neighbors, compute Jaccard
-		DataSet<Edge<Long, Double>> edgesWithJaccardValues =
-				graphWithVertexValues.getTriplets().map(new ComputeJaccard());
-
-		// emit result
-		if (fileOutput) {
-			edgesWithJaccardValues.writeAsCsv(outputPath, "\n", ",");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute("Executing Jaccard Similarity Measure");
-		} else {
-			edgesWithJaccardValues.print();
-		}
-
-	}
-
-	@Override
-	public String getDescription() {
-		return "Vertex Jaccard Similarity Measure";
-	}
-
-	/**
-	 * Each vertex will have a HashSet containing its neighbor ids as value.
-	 */
-	private static final class GatherNeighbors implements ReduceNeighborsFunction<HashSet<Long>> {
-
-		@Override
-		public HashSet<Long> reduceNeighbors(HashSet<Long> first, HashSet<Long> second) {
-			first.addAll(second);
-			return new HashSet<Long>(first);
-		}
-	}
-
-	/**
-	 * The edge weight will be the Jaccard coefficient, which is computed as follows:
-	 *
-	 * Consider the edge x-y
-	 * We denote by sizeX and sizeY, the neighbors hash set size of x and y respectively.
-	 * sizeX+sizeY = union + intersection of neighborhoods
-	 * size(hashSetX.addAll(hashSetY)).distinct = union of neighborhoods
-	 * The intersection can then be deduced.
-	 *
-	 * The Jaccard similarity coefficient is then, the intersection/union.
-	 */
-	private static final class ComputeJaccard implements
-			MapFunction<Triplet<Long, HashSet<Long>, Double>, Edge<Long, Double>> {
-
-		@Override
-		public Edge<Long, Double> map(Triplet<Long, HashSet<Long>, Double> triplet) throws Exception {
-
-			Vertex<Long, HashSet<Long>> srcVertex = triplet.getSrcVertex();
-			Vertex<Long, HashSet<Long>> trgVertex = triplet.getTrgVertex();
-
-			Long x = srcVertex.getId();
-			Long y = trgVertex.getId();
-			HashSet<Long> neighborSetY = trgVertex.getValue();
-
-			double unionPlusIntersection = srcVertex.getValue().size() + neighborSetY.size();
-			// within a HashSet, all elements are distinct
-			HashSet<Long> unionSet = new HashSet<Long>();
-			unionSet.addAll(srcVertex.getValue());
-			unionSet.addAll(neighborSetY);
-			double union = unionSet.size();
-			double intersection = unionPlusIntersection - union;
-
-			return new Edge<Long, Double>(x, y, intersection/union);
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String edgeInputPath = null;
-	private static String outputPath = null;
-
-	private static boolean parseParameters(String [] args) {
-		if(args.length > 0) {
-			if(args.length != 2) {
-				System.err.println("Usage JaccardSimilarityMeasure <edge path> <output path>");
-				return false;
-			}
-
-			fileOutput = true;
-			edgeInputPath = args[0];
-			outputPath = args[1];
-		} else {
-			System.out.println("Executing JaccardSimilarityMeasure example with default parameters and built-in default data.");
-			System.out.println("Provide parameters to read input data from files.");
-			System.out.println("Usage JaccardSimilarityMeasure <edge path> <output path>");
-		}
-
-		return true;
-	}
-
-	private static DataSet<Edge<Long, Double>> 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, Double>>() {
-						@Override
-						public Edge<Long, Double> map(Tuple2<Long, Long> tuple2) throws Exception {
-							return new Edge<Long, Double>(tuple2.f0, tuple2.f1, new Double(0));
-						}
-					});
-		} else {
-			return JaccardSimilarityMeasureData.getDefaultEdgeDataSet(env);
-		}
-	}
-}


[08/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
deleted file mode 100644
index e347bc5..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
+++ /dev/null
@@ -1,302 +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.example;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-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.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.EdgesFunctionWithVertexValue;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.MusicProfilesData;
-import org.apache.flink.graph.library.LabelPropagation;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.util.Collector;
-
-/**
- * This example demonstrates how to mix the DataSet Flink API with the Gelly API.
- * The input is a set <userId - songId - playCount> triplets and
- * a set of bad records, i.e. song ids that should not be trusted.
- * Initially, we use the DataSet API to filter out the bad records.
- * Then, we use Gelly to create a user -> song weighted bipartite graph and compute
- * the top song (most listened) per user.
- * Then, we use the DataSet API again, to create a user-user similarity graph,
- * based on common songs, where users that are listeners of the same song
- * are connected. A user-defined threshold on the playcount value
- * defines when a user is considered to be a listener of a song.
- * Finally, we use the graph API to run the label propagation community detection algorithm on
- * the similarity graph.
- *
- * The triplets input is expected to be given as one triplet per line,
- * in the following format: "<userID>\t<songID>\t<playcount>".
- *
- * The mismatches input file is expected to contain one mismatch record per line,
- * in the following format:
- * "ERROR: <songID trackID> song_title"
- *
- * If no arguments are provided, the example runs with default data from {@link MusicProfilesData}.
- */
-@SuppressWarnings("serial")
-public class MusicProfiles implements ProgramDescription {
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		/**
-		 * Read the user-song-play triplets.
-		 */
-		DataSet<Tuple3<String, String, Integer>> triplets = getUserSongTripletsData(env);
-
-		/**
-		 * Read the mismatches dataset and extract the songIDs
-		 */
-		DataSet<Tuple1<String>> mismatches = getMismatchesData(env).map(new ExtractMismatchSongIds());
-
-		/**
-		 * Filter out the mismatches from the triplets dataset
-		 */
-		DataSet<Tuple3<String, String, Integer>> validTriplets = triplets
-				.coGroup(mismatches).where(1).equalTo(0)
-				.with(new FilterOutMismatches());
-
-		/**
-		 * Create a user -> song weighted bipartite graph where the edge weights
-		 * correspond to play counts
-		 */
-		Graph<String, NullValue, Integer> userSongGraph = Graph.fromTupleDataSet(validTriplets, env);
-
-		/**
-		 * Get the top track (most listened) for each user
-		 */
-		DataSet<Tuple2<String, String>> usersWithTopTrack = userSongGraph
-				.groupReduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT)
-				.filter(new FilterSongNodes());
-
-		if (fileOutput) {
-			usersWithTopTrack.writeAsCsv(topTracksOutputPath, "\n", "\t");
-		} else {
-			usersWithTopTrack.print();
-		}
-
-		/**
-		 * Create a user-user similarity graph, based on common songs, i.e. two
-		 * users that listen to the same song are connected. For each song, we
-		 * create an edge between each pair of its in-neighbors.
-		 */
-		DataSet<Edge<String, NullValue>> similarUsers = userSongGraph
-				.getEdges()
-				// filter out user-song edges that are below the playcount threshold
-				.filter(new FilterFunction<Edge<String, Integer>>() {
-					public boolean filter(Edge<String, Integer> edge) {
-						return (edge.getValue() > playcountThreshold);
-					}
-				}).groupBy(1)
-				.reduceGroup(new CreateSimilarUserEdges()).distinct();
-
-		Graph<String, Long, NullValue> similarUsersGraph = Graph.fromDataSet(similarUsers,
-				new MapFunction<String, Long>() {
-					public Long map(String value) {
-						return 1l;
-					}
-				}, env).getUndirected();
-
-		/**
-		 * Detect user communities using the label propagation library method
-		 */
-		// Initialize each vertex with a unique numeric label and run the label propagation algorithm
-		DataSet<Tuple2<String, Long>> idsWithInitialLabels = DataSetUtils
-				.zipWithUniqueId(similarUsersGraph.getVertexIds())
-				.map(new MapFunction<Tuple2<Long, String>, Tuple2<String, Long>>() {
-					@Override
-					public Tuple2<String, Long> map(Tuple2<Long, String> tuple2) throws Exception {
-						return new Tuple2<String, Long>(tuple2.f1, tuple2.f0);
-					}
-				});
-
-		DataSet<Vertex<String, Long>> verticesWithCommunity = similarUsersGraph
-				.joinWithVertices(idsWithInitialLabels,
-						new MapFunction<Tuple2<Long, Long>, Long>() {
-							public Long map(Tuple2<Long, Long> value) {
-								return value.f1;
-							}
-						}).run(new LabelPropagation<String, NullValue>(maxIterations));
-
-		if (fileOutput) {
-			verticesWithCommunity.writeAsCsv(communitiesOutputPath, "\n", "\t");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute();
-		} else {
-			verticesWithCommunity.print();
-		}
-
-	}
-
-	public static final class ExtractMismatchSongIds implements MapFunction<String, Tuple1<String>> {
-
-		public Tuple1<String> map(String value) {
-			String[] tokens = value.split("\\s+");
-			String songId = tokens[1].substring(1);
-			return new Tuple1<String>(songId);
-		}
-	}
-
-	public static final class FilterOutMismatches implements CoGroupFunction<Tuple3<String, String, Integer>,
-		Tuple1<String>, Tuple3<String, String, Integer>> {
-
-		public void coGroup(Iterable<Tuple3<String, String, Integer>> triplets,
-				Iterable<Tuple1<String>> invalidSongs, Collector<Tuple3<String, String, Integer>> out) {
-
-			if (!invalidSongs.iterator().hasNext()) {
-				// this is a valid triplet
-				for (Tuple3<String, String, Integer> triplet : triplets) {
-					out.collect(triplet);
-				}
-			}
-		}
-	}
-
-	public static final class FilterSongNodes implements FilterFunction<Tuple2<String, String>> {
-		public boolean filter(Tuple2<String, String> value) throws Exception {
-			return !value.f1.equals("");
-		}
-	}
-
-	public static final class GetTopSongPerUser	implements EdgesFunctionWithVertexValue<String, NullValue, Integer,
-		Tuple2<String, String>> {
-
-		public void iterateEdges(Vertex<String, NullValue> vertex,
-				Iterable<Edge<String, Integer>> edges, Collector<Tuple2<String, String>> out) throws Exception {
-
-			int maxPlaycount = 0;
-			String topSong = "";
-			for (Edge<String, Integer> edge : edges) {
-				if (edge.getValue() > maxPlaycount) {
-					maxPlaycount = edge.getValue();
-					topSong = edge.getTarget();
-				}
-			}
-			out.collect(new Tuple2<String, String>(vertex.getId(), topSong));
-		}
-	}
-
-	public static final class CreateSimilarUserEdges implements GroupReduceFunction<Edge<String, Integer>,
-		Edge<String, NullValue>> {
-
-		public void reduce(Iterable<Edge<String, Integer>> edges, Collector<Edge<String, NullValue>> out) {
-			List<String> listeners = new ArrayList<String>();
-			for (Edge<String, Integer> edge : edges) {
-				listeners.add(edge.getSource());
-			}
-			for (int i = 0; i < listeners.size() - 1; i++) {
-				for (int j = i + 1; j < listeners.size(); j++) {
-					out.collect(new Edge<String, NullValue>(listeners.get(i),
-							listeners.get(j), NullValue.getInstance()));
-				}
-			}
-		}
-	}
-
-	@Override
-	public String getDescription() {
-		return "Music Profiles Example";
-	}
-
-	// ******************************************************************************************************************
-	// UTIL METHODS
-	// ******************************************************************************************************************
-
-	private static boolean fileOutput = false;
-
-	private static String userSongTripletsInputPath = null;
-
-	private static String mismatchesInputPath = null;
-
-	private static String topTracksOutputPath = null;
-
-	private static int playcountThreshold = 0;
-
-	private static String communitiesOutputPath = null;
-
-	private static int maxIterations = 10;
-
-	private static boolean parseParameters(String[] args) {
-
-		if(args.length > 0) {
-			if(args.length != 6) {
-				System.err.println("Usage: MusicProfiles <input user song triplets path>" +
-						" <input song mismatches path> <output top tracks path> "
-						+ "<playcount threshold> <output communities path> <num iterations>");
-				return false;
-			}
-
-			fileOutput = true;
-			userSongTripletsInputPath = args[0];
-			mismatchesInputPath = args[1];
-			topTracksOutputPath = args[2];
-			playcountThreshold = Integer.parseInt(args[3]);
-			communitiesOutputPath = args[4];
-			maxIterations = Integer.parseInt(args[5]);
-		} else {
-			System.out.println("Executing Music Profiles example with default parameters and built-in default data.");
-			System.out.println("  Provide parameters to read input data from files.");
-			System.out.println("  See the documentation for the correct format of input files.");
-			System.out.println("Usage: MusicProfiles <input user song triplets path>" +
-					" <input song mismatches path> <output top tracks path> "
-					+ "<playcount threshold> <output communities path> <num iterations>");
-		}
-		return true;
-	}
-
-	private static DataSet<Tuple3<String, String, Integer>> getUserSongTripletsData(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(userSongTripletsInputPath)
-					.lineDelimiter("\n").fieldDelimiter("\t")
-					.types(String.class, String.class, Integer.class);
-		} else {
-			return MusicProfilesData.getUserSongTriplets(env);
-		}
-	}
-
-	private static DataSet<String> getMismatchesData(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readTextFile(mismatchesInputPath);
-		} else {
-			return MusicProfilesData.getMismatches(env);
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
deleted file mode 100644
index ef09bff..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/SingleSourceShortestPaths.java
+++ /dev/null
@@ -1,199 +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.example;
-
-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.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
-
-/**
- * This example shows how to use Gelly's vertex-centric iterations.
- * 
- * It is an implementation of the Single-Source-Shortest-Paths algorithm.
- * For a gather-sum-apply implementation of the same algorithm, please refer to {@link GSASingleSourceShortestPaths}. 
- *
- * The input file is a plain text file and must be formatted as follows:
- * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
- * separated by tabs. Edges themselves are separated by newlines.
- * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
- * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
- *
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.graph.example.utils.SingleSourceShortestPathsData}
- */
-public class SingleSourceShortestPaths implements ProgramDescription {
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Edge<Long, Double>> edges = getEdgesDataSet(env);
-
-		Graph<Long, Double, Double> graph = Graph.fromDataSet(edges, new InitVertices(srcVertexId), env);
-
-		// Execute the vertex-centric iteration
-		Graph<Long, Double, Double> result = graph.runVertexCentricIteration(
-				new VertexDistanceUpdater(), new MinDistanceMessenger(), maxIterations);
-
-		// Extract the vertices as the result
-		DataSet<Vertex<Long, Double>> singleSourceShortestPaths = result.getVertices();
-
-		// emit result
-		if (fileOutput) {
-			singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",");
-
-			// since file sinks are lazy, we trigger the execution explicitly
-			env.execute("Single Source Shortest Paths Example");
-		} else {
-			singleSourceShortestPaths.print();
-		}
-
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Single Source Shortest Path UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	private static final class InitVertices implements MapFunction<Long, Double>{
-
-		private long srcId;
-
-		public InitVertices(long srcId) {
-			this.srcId = srcId;
-		}
-
-		public Double map(Long id) {
-			if (id.equals(srcId)) {
-				return 0.0;
-			}
-			else {
-				return Double.POSITIVE_INFINITY;
-			}
-		}
-	}
-
-	/**
-	 * Function that updates the value of a vertex by picking the minimum
-	 * distance from all incoming messages.
-	 */
-	@SuppressWarnings("serial")
-	public static final class VertexDistanceUpdater extends VertexUpdateFunction<Long, Double, Double> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Double> vertex, MessageIterator<Double> inMessages) {
-
-			Double minDistance = Double.MAX_VALUE;
-
-			for (double msg : inMessages) {
-				if (msg < minDistance) {
-					minDistance = msg;
-				}
-			}
-
-			if (vertex.getValue() > minDistance) {
-				setNewVertexValue(minDistance);
-			}
-		}
-	}
-
-	/**
-	 * Distributes the minimum distance associated with a given vertex among all
-	 * the target vertices summed up with the edge's value.
-	 */
-	@SuppressWarnings("serial")
-	public static final class MinDistanceMessenger extends MessagingFunction<Long, Double, Double, Double> {
-
-		@Override
-		public void sendMessages(Vertex<Long, Double> vertex) {
-			for (Edge<Long, Double> edge : getEdges()) {
-				sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue());
-			}
-		}
-	}
-
-	// ******************************************************************************************************************
-	// UTIL METHODS
-	// ******************************************************************************************************************
-
-	private static boolean fileOutput = false;
-
-	private static Long srcVertexId = 1l;
-
-	private static String edgesInputPath = null;
-
-	private static String outputPath = null;
-
-	private static int maxIterations = 5;
-
-	private static boolean parseParameters(String[] args) {
-
-		if(args.length > 0) {
-			if(args.length != 4) {
-				System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-						" <input edges path> <output path> <num iterations>");
-				return false;
-			}
-
-			fileOutput = true;
-			srcVertexId = Long.parseLong(args[0]);
-			edgesInputPath = args[1];
-			outputPath = args[2];
-			maxIterations = Integer.parseInt(args[3]);
-		} else {
-				System.out.println("Executing Single Source Shortest Paths example "
-						+ "with default parameters and built-in default data.");
-				System.out.println("  Provide parameters to read input data from files.");
-				System.out.println("  See the documentation for the correct format of input files.");
-				System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-						" <input edges path> <output path> <num iterations>");
-		}
-		return true;
-	}
-
-	private static DataSet<Edge<Long, Double>> getEdgesDataSet(ExecutionEnvironment env) {
-		if (fileOutput) {
-			return env.readCsvFile(edgesInputPath)
-					.lineDelimiter("\n")
-					.fieldDelimiter("\t")
-					.types(Long.class, Long.class, Double.class)
-					.map(new Tuple3ToEdgeMap<Long, Double>());
-		} else {
-			return SingleSourceShortestPathsData.getDefaultEdgeDataSet(env);
-		}
-	}
-
-	@Override
-	public String getDescription() {
-		return "Vertex-centric Single Source Shortest Paths";
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
deleted file mode 100644
index c37b2b5..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/CommunityDetectionData.java
+++ /dev/null
@@ -1,95 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides the default data set used for the Simple Community Detection test program.
- * If no parameters are given to the program, the default edge data set is used.
- */
-public class CommunityDetectionData {
-
-	// the algorithm is not guaranteed to always converge
-	public static final Integer MAX_ITERATIONS = 30;
-
-	public static final double DELTA = 0.5f;
-
-	public static final String COMMUNITIES_SINGLE_ITERATION = "1,5\n" + "2,6\n"
-			+ "3,1\n" + "4,1\n" + "5,1\n" + "6,8\n" + "7,8\n" + "8,7"; 
-
-	public static final String COMMUNITIES_WITH_TIE = "1,2\n" + "2,1\n" + "3,1\n" + "4,1\n" + "5,1";
-
-	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 3L, 4.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 5.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 6.0));
-		edges.add(new Edge<Long, Double>(5L, 6L, 7.0));
-		edges.add(new Edge<Long, Double>(5L, 7L, 8.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 9.0));
-		edges.add(new Edge<Long, Double>(7L, 12L, 10.0));
-		edges.add(new Edge<Long, Double>(8L, 9L, 11.0));
-		edges.add(new Edge<Long, Double>(8L, 10L, 12.0));
-		edges.add(new Edge<Long, Double>(8L, 11L, 13.0));
-		edges.add(new Edge<Long, Double>(9L, 10L, 14.0));
-		edges.add(new Edge<Long, Double>(9L, 11L, 15.0));
-		edges.add(new Edge<Long, Double>(10L, 11L, 16.0));
-		edges.add(new Edge<Long, Double>(10L, 12L, 17.0));
-		edges.add(new Edge<Long, Double>(11L, 12L, 18.0));
-
-		return env.fromCollection(edges);
-	}
-
-	public static DataSet<Edge<Long, Double>> getSimpleEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 2.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(1L, 5L, 4.0));
-		edges.add(new Edge<Long, Double>(2L, 6L, 5.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 6.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 7.0));
-		edges.add(new Edge<Long, Double>(7L, 8L, 8.0));
-
-		return env.fromCollection(edges);
-	}
-
-	private CommunityDetectionData() {}
-
-	public static DataSet<Edge<Long, Double>> getTieEdgeDataSet(ExecutionEnvironment env) {
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 5L, 1.0));
-
-		return env.fromCollection(edges);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
deleted file mode 100644
index 67864eb..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ConnectedComponentsDefaultData.java
+++ /dev/null
@@ -1,57 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.types.NullValue;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the connected components example program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class ConnectedComponentsDefaultData {
-
-	public static final Integer MAX_ITERATIONS = 4;
-
-	public static final String EDGES = "1	2\n" + "2	3\n" + "2	4\n" + "3	4";
-
-	public static final Object[][] DEFAULT_EDGES = new Object[][] {
-		new Object[]{1L, 2L},
-		new Object[]{2L, 3L},
-		new Object[]{2L, 4L},
-		new Object[]{3L, 4L}
-	};
-
-	public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		List<Edge<Long, NullValue>> edgeList = new LinkedList<Edge<Long, NullValue>>();
-		for (Object[] edge : DEFAULT_EDGES) {
-			edgeList.add(new Edge<Long, NullValue>((Long) edge[0], (Long) edge[1], NullValue.getInstance()));
-		}
-		return env.fromCollection(edgeList);
-	}
-
-	public static final String VERTICES_WITH_MIN_ID = "1,1\n" + "2,1\n" + "3,1\n" + "4,1";
-
-	private ConnectedComponentsDefaultData() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
deleted file mode 100644
index 80765bf..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/EuclideanGraphData.java
+++ /dev/null
@@ -1,86 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.EuclideanGraphWeighing;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the Euclidean Graph example program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class EuclideanGraphData {
-
-	public static final int NUM_VERTICES = 9;
-
-	public static final String VERTICES = "1,1.0,1.0\n" + "2,2.0,2.0\n" + "3,3.0,3.0\n" + "4,4.0,4.0\n" + "5,5.0,5.0\n" +
-			"6,6.0,6.0\n" + "7,7.0,7.0\n" + "8,8.0,8.0\n" + "9,9.0,9.0";
-
-	public static DataSet<Vertex<Long, EuclideanGraphWeighing.Point>> getDefaultVertexDataSet(ExecutionEnvironment env) {
-
-		List<Vertex<Long, EuclideanGraphWeighing.Point>> vertices = new ArrayList<Vertex<Long, EuclideanGraphWeighing.Point>>();
-		for(int i=1; i<=NUM_VERTICES; i++) {
-			vertices.add(new Vertex<Long, EuclideanGraphWeighing.Point>(new Long(i),
-					new EuclideanGraphWeighing.Point(new Double(i), new Double(i))));
-		}
-
-		return env.fromCollection(vertices);
-	}
-
-	public static final String EDGES = "1,2\n" + "1,4\n" + "2,3\n" + "2,4\n" + "2,5\n" +
-			"3,5\n" + "4,5\n" + "4,6\n" + "5,7\n" + "5,9\n" + "6,7\n" + "6,8\n" +
-			"7,8\n" + "7,9\n" +  "8,9";
-
-	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 0.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 3L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 0.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 0.0));
-		edges.add(new Edge<Long, Double>(4L, 6L, 0.0));
-		edges.add(new Edge<Long, Double>(5L, 7L, 0.0));
-		edges.add(new Edge<Long, Double>(5L, 9L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 7L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(6L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(7L, 8L, 0.0));
-		edges.add(new Edge<Long, Double>(7L, 9L, 0.0));
-		edges.add(new Edge<Long, Double>(8L, 9L, 0.0));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final String RESULTED_WEIGHTED_EDGES = "1,2,1.4142135623730951\n" + "1,4,4.242640687119285\n" +
-			"2,3,1.4142135623730951\n" + "2,4,2.8284271247461903\n" + "2,5,4.242640687119285\n" + "3,5,2.8284271247461903\n" +
-			"4,5,1.4142135623730951\n" + "4,6,2.8284271247461903\n" + "5,7,2.8284271247461903\n" + "5,9,5.656854249492381\n" +
-			"6,7,1.4142135623730951\n" + "6,8,2.8284271247461903\n" + "7,8,1.4142135623730951\n" + "7,9,2.8284271247461903\n" +
-			"8,9,1.4142135623730951";
-
-	private EuclideanGraphData() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
deleted file mode 100644
index 7fbee46..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/ExampleUtils.java
+++ /dev/null
@@ -1,162 +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.example.utils;
-
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.util.Collector;
-
-public class ExampleUtils {
-
-	@SuppressWarnings({ "serial", "unchecked", "rawtypes" })
-	public static void printResult(DataSet set, String msg) {
-		set.output(new PrintingOutputFormatWithMessage(msg) {
-		});
-	}
-
-	public static class PrintingOutputFormatWithMessage<T> implements
-			OutputFormat<T> {
-
-		private static final long serialVersionUID = 1L;
-
-		private transient PrintStream stream;
-
-		private transient String prefix;
-
-		private String message;
-
-		// --------------------------------------------------------------------------------------------
-
-		/**
-		 * Instantiates a printing output format that prints to standard out.
-		 */
-		public PrintingOutputFormatWithMessage() {
-		}
-
-		public PrintingOutputFormatWithMessage(String msg) {
-			this.message = msg;
-		}
-
-		@Override
-		public void open(int taskNumber, int numTasks) {
-			// get the target stream
-			this.stream = System.out;
-
-			// set the prefix to message
-			this.prefix = message + ": ";
-		}
-
-		@Override
-		public void writeRecord(T record) {
-			if (this.prefix != null) {
-				this.stream.println(this.prefix + record.toString());
-			} else {
-				this.stream.println(record.toString());
-			}
-		}
-
-		@Override
-		public void close() {
-			this.stream = null;
-			this.prefix = null;
-		}
-
-		@Override
-		public String toString() {
-			return "Print to System.out";
-		}
-
-		@Override
-		public void configure(Configuration parameters) {
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static DataSet<Vertex<Long, NullValue>> getVertexIds(
-			ExecutionEnvironment env, final long numVertices) {
-		return env.generateSequence(1, numVertices).map(
-				new MapFunction<Long, Vertex<Long, NullValue>>() {
-					public Vertex<Long, NullValue> map(Long l) {
-						return new Vertex<Long, NullValue>(l, NullValue
-								.getInstance());
-					}
-				});
-	}
-
-	@SuppressWarnings("serial")
-	public static DataSet<Edge<Long, NullValue>> getRandomEdges(
-			ExecutionEnvironment env, final long numVertices) {
-		return env.generateSequence(1, numVertices).flatMap(
-				new FlatMapFunction<Long, Edge<Long, NullValue>>() {
-					@Override
-					public void flatMap(Long key, Collector<Edge<Long, NullValue>> out) throws Exception {
-						int numOutEdges = (int) (Math.random() * (numVertices / 2));
-						for (int i = 0; i < numOutEdges; i++) {
-							long target = (long) (Math.random() * numVertices) + 1;
-							out.collect(new Edge<Long, NullValue>(key, target,
-									NullValue.getInstance()));
-						}
-					}
-				});
-	}
-
-	public static DataSet<Vertex<Long, Double>> getLongDoubleVertexData(
-			ExecutionEnvironment env) {
-		List<Vertex<Long, Double>> vertices = new ArrayList<Vertex<Long, Double>>();
-		vertices.add(new Vertex<Long, Double>(1L, 1.0));
-		vertices.add(new Vertex<Long, Double>(2L, 2.0));
-		vertices.add(new Vertex<Long, Double>(3L, 3.0));
-		vertices.add(new Vertex<Long, Double>(4L, 4.0));
-		vertices.add(new Vertex<Long, Double>(5L, 5.0));
-
-		return env.fromCollection(vertices);
-	}
-
-	public static DataSet<Edge<Long, Double>> getLongDoubleEdgeData(
-			ExecutionEnvironment env) {
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, 12.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 13.0));
-		edges.add(new Edge<Long, Double>(2L, 3L, 23.0));
-		edges.add(new Edge<Long, Double>(3L, 4L, 34.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 35.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 45.0));
-		edges.add(new Edge<Long, Double>(5L, 1L, 51.0));
-
-		return env.fromCollection(edges);
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private ExampleUtils() {
-		throw new RuntimeException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
deleted file mode 100644
index 7b69ec0..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/IncrementalSSSPData.java
+++ /dev/null
@@ -1,95 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Vertex;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the IncrementalSSSP example program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class IncrementalSSSPData {
-
-	public static final int NUM_VERTICES = 5;
-
-	public static final String VERTICES = "1,6.0\n" + "2,2.0\n" + "3,3.0\n" + "4,1.0\n" + "5,0.0";
-
-	public static DataSet<Vertex<Long, Double>> getDefaultVertexDataSet(ExecutionEnvironment env) {
-
-		List<Vertex<Long, Double>> vertices = new ArrayList<Vertex<Long, Double>>();
-		vertices.add(new Vertex<Long, Double>(1L, 6.0));
-		vertices.add(new Vertex<Long, Double>(2L, 2.0));
-		vertices.add(new Vertex<Long, Double>(3L, 3.0));
-		vertices.add(new Vertex<Long, Double>(4L, 1.0));
-		vertices.add(new Vertex<Long, Double>(5L, 0.0));
-
-		return env.fromCollection(vertices);
-	}
-
-	public static final String EDGES = "1,3,3.0\n" + "2,4,3.0\n" + "2,5,2.0\n" + "3,2,1.0\n" + "3,5,5.0\n" +
-			"4,5,1.0";
-
-	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 4L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
-		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 5.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final String EDGES_IN_SSSP = "1,3,3.0\n" + "2,5,2.0\n" + "3,2,1.0\n" + "4,5,1.0";
-
-	public static final DataSet<Edge<Long, Double>> getDefaultEdgesInSSSP(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 3L, 3.0));
-		edges.add(new Edge<Long, Double>(2L, 5L, 2.0));
-		edges.add(new Edge<Long, Double>(3L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 5L, 1.0));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final String SRC_EDGE_TO_BE_REMOVED = "2";
-
-	public static final String TRG_EDGE_TO_BE_REMOVED = "5";
-
-	public static final String VAL_EDGE_TO_BE_REMOVED = "2.0";
-
-	public static final Edge<Long, Double> getDefaultEdgeToBeRemoved() {
-
-		return new Edge<Long, Double>(2L, 5L, 2.0);
-	}
-
-	public static final String RESULTED_VERTICES = "1," + Double.MAX_VALUE + "\n" + "2," + Double.MAX_VALUE+ "\n"
-			+ "3," + Double.MAX_VALUE + "\n" + "4,1.0\n" + "5,0.0";
-
-	private IncrementalSSSPData() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
deleted file mode 100644
index 7564b95..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/JaccardSimilarityMeasureData.java
+++ /dev/null
@@ -1,58 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the Jaccard Similarity Measure example program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class JaccardSimilarityMeasureData {
-
-	public static final String EDGES = "1	2\n" + "1	3\n" + "1	4\n" + "1	5\n" + "2	3\n" + "2	4\n" +
-			"2	5\n" + "3	4\n" + "3	5\n" + "4	5";
-
-	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(1L, 2L, new Double(0)));
-		edges.add(new Edge<Long, Double>(1L, 3L, new Double(0)));
-		edges.add(new Edge<Long, Double>(1L, 4L, new Double(0)));
-		edges.add(new Edge<Long, Double>(1L, 5L, new Double(0)));
-		edges.add(new Edge<Long, Double>(2L, 3L, new Double(0)));
-		edges.add(new Edge<Long, Double>(2L, 4L, new Double(0)));
-		edges.add(new Edge<Long, Double>(2L, 5L, new Double(0)));
-		edges.add(new Edge<Long, Double>(3L, 4L, new Double(0)));
-		edges.add(new Edge<Long, Double>(3L, 5L, new Double(0)));
-		edges.add(new Edge<Long, Double>(4L, 5L, new Double(0)));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final String JACCARD_EDGES = "1,2,0.6\n" + "1,3,0.6\n" + "1,4,0.6\n" + "1,5,0.6\n" +
-			"2,3,0.6\n" + "2,4,0.6\n" + "2,5,0.6\n" + "3,4,0.6\n" + "3,5,0.6\n" + "4,5,0.6";
-
-	private JaccardSimilarityMeasureData() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
deleted file mode 100644
index 0a92097..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/LabelPropagationData.java
+++ /dev/null
@@ -1,114 +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.example.utils;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.NullValue;
-
-/**
- * Provides the default data set used for the Label Propagation test program.
- * If no parameters are given to the program, the default edge data set is used.
- */
-public class LabelPropagationData {
-	
-	public static final String LABELS_AFTER_1_ITERATION = "1,10\n" +
-			"2,10\n" +
-			"3,10\n" +
-			"4,40\n" +
-			"5,40\n" +
-			"6,40\n" +
-			"7,40\n";
-
-	public static final String LABELS_WITH_TIE ="1,10\n" +
-			"2,10\n" +
-			"3,10\n" +
-			"4,10\n" +
-			"5,20\n" +
-			"6,20\n" +
-			"7,20\n" +
-			"8,20\n" +
-			"9,20\n";
-
-	private LabelPropagationData() {}
-
-	public static final DataSet<Vertex<Long, Long>> getDefaultVertexSet(ExecutionEnvironment env) {
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1l, 10l));
-		vertices.add(new Vertex<Long, Long>(2l, 10l));
-		vertices.add(new Vertex<Long, Long>(3l, 30l));
-		vertices.add(new Vertex<Long, Long>(4l, 40l));
-		vertices.add(new Vertex<Long, Long>(5l, 40l));
-		vertices.add(new Vertex<Long, Long>(6l, 40l));
-		vertices.add(new Vertex<Long, Long>(7l, 40l));
-
-		return env.fromCollection(vertices);
-	}
-
-	public static final DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(5L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(7L, 3L, NullValue.getInstance()));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final DataSet<Vertex<Long, Long>> getTieVertexSet(ExecutionEnvironment env) {
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1l, 10l));
-		vertices.add(new Vertex<Long, Long>(2l, 10l));
-		vertices.add(new Vertex<Long, Long>(3l, 10l));
-		vertices.add(new Vertex<Long, Long>(4l, 10l));
-		vertices.add(new Vertex<Long, Long>(5l, 0l));
-		vertices.add(new Vertex<Long, Long>(6l, 20l));
-		vertices.add(new Vertex<Long, Long>(7l, 20l));
-		vertices.add(new Vertex<Long, Long>(8l, 20l));
-		vertices.add(new Vertex<Long, Long>(9l, 20l));
-
-		return env.fromCollection(vertices);
-	}
-
-	public static final DataSet<Edge<Long, NullValue>> getTieEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(5L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(7L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(8L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(9L, 5L, NullValue.getInstance()));
-
-		return env.fromCollection(edges);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
deleted file mode 100644
index 3a97d1f..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/MusicProfilesData.java
+++ /dev/null
@@ -1,108 +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.example.utils;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-
-/**
- * Provides the default data sets used for the Music Profiles example program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class MusicProfilesData {
-
-	public static DataSet<Tuple3<String, String, Integer>> getUserSongTriplets(ExecutionEnvironment env) {
-		List<Tuple3<String, String, Integer>> triplets = new ArrayList<Tuple3<String, String, Integer>>();
-		
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_2", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_3", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_4", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_1", "song_5", 1));
-		
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_6", 40));
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_7", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_2", "song_8", 3));
-		
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_2", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_3", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_8", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_9", 1));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_10", 8));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_11", 90));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_12", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_13", 34));
-		triplets.add(new Tuple3<String, String, Integer>("user_3", "song_14", 17));
-		
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_1", 100));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_6", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_8", 20));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_12", 30));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_13", 1));
-		triplets.add(new Tuple3<String, String, Integer>("user_4", "song_15", 1));
-		
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_3", 300));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_4", 4));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_5", 5));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_8", 8));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_9", 9));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_10", 10));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_12", 12));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_13", 13));
-		triplets.add(new Tuple3<String, String, Integer>("user_5", "song_15", 15));
-
-		triplets.add(new Tuple3<String, String, Integer>("user_6", "song_6", 30));
-
-		return env.fromCollection(triplets);
-	}
-	
-	public static DataSet<String> getMismatches(ExecutionEnvironment env) {
-		List<String> errors = new ArrayList<String>();
-		errors.add("ERROR: <song_8 track_8> Sever");
-		errors.add("ERROR: <song_15 track_15> Black Trees");
-		return env.fromCollection(errors);
-	}
-
-	public static final String USER_SONG_TRIPLETS = "user_1	song_1	100\n" + "user_1	song_5	200\n"
-			+ "user_2	song_1	10\n" + "user_2	song_4	20\n"
-			+ "user_3	song_2	3\n"
-			+ "user_4	song_2	1\n" + "user_4	song_3	2\n"
-			+ "user_5	song_3	30";
-
-	public static final String MISMATCHES = "ERROR: <song_5 track_8> Angie";
-
-	public static final String MAX_ITERATIONS = "2";
-
-	public static final String TOP_SONGS_RESULT = "user_1	song_1\n" +
-								"user_2	song_4\n" +
-								"user_3	song_2\n" +
-								"user_4	song_3\n" +
-								"user_5	song_3";
-
-	public static final String COMMUNITIES_RESULT = "user_1	1\n" +
-								"user_2	1\n" +
-								"user_3	3\n" +
-								"user_4	3\n" +
-								"user_5	4";
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
deleted file mode 100644
index 58d4f8b..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/PageRankData.java
+++ /dev/null
@@ -1,69 +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.example.utils;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-
-/**
- * Provides the default data set used for the PageRank test program.
- * If no parameters are given to the program, the default edge data set is used.
- */
-public class PageRankData {
-	
-	public static final String EDGES = "2	1\n" +
-										"5	2\n" + 
-										"5	4\n" +
-										"4	3\n" +
-										"4	2\n" +
-										"1	4\n" +
-										"1	2\n" +
-										"1	3\n" +
-										"3	5\n";
-
-	
-	public static final String RANKS_AFTER_3_ITERATIONS = "1,0.237\n" +
-														"2,0.248\n" + 
-														"3,0.173\n" +
-														"4,0.175\n" +
-														"5,0.165\n";
-
-	private PageRankData() {}
-
-	public static final DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, Double>> edges = new ArrayList<Edge<Long, Double>>();
-		edges.add(new Edge<Long, Double>(2L, 1L, 1.0));
-		edges.add(new Edge<Long, Double>(5L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(5L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(4L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 4L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 2L, 1.0));
-		edges.add(new Edge<Long, Double>(1L, 3L, 1.0));
-		edges.add(new Edge<Long, Double>(3L, 5L, 1.0));
-
-		return env.fromCollection(edges);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
deleted file mode 100644
index 6b985c5..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/SingleSourceShortestPathsData.java
+++ /dev/null
@@ -1,62 +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.example.utils;
-
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-
-/**
- * Provides the default data set used for the Single Source Shortest Paths example program.
- * If no parameters are given to the program, the default edge data set is used.
- */
-public class SingleSourceShortestPathsData {
-
-	public static final Long SRC_VERTEX_ID = 1L;
-
-	public static final String EDGES = "1\t2\t12.0\n" + "1\t3\t13.0\n" + "2\t3\t23.0\n" + "3\t4\t34.0\n" + "3\t5\t35.0\n" +
-					"4\t5\t45.0\n" + "5\t1\t51.0";
-
-	public static final Object[][] DEFAULT_EDGES = new Object[][] {
-		new Object[]{1L, 2L, 12.0},
-		new Object[]{1L, 3L, 13.0},
-		new Object[]{2L, 3L, 23.0},
-		new Object[]{3L, 4L, 34.0},
-		new Object[]{3L, 5L, 35.0},
-		new Object[]{4L, 5L, 45.0},
-		new Object[]{5L, 1L, 51.0}
-	};
-
-	public static final String RESULTED_SINGLE_SOURCE_SHORTEST_PATHS =  "1,0.0\n" + "2,12.0\n" + "3,13.0\n" + 
-								"4,47.0\n" + "5,48.0";
-
-	public static DataSet<Edge<Long, Double>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-		
-		List<Edge<Long, Double>> edgeList = new LinkedList<Edge<Long, Double>>();
-		for (Object[] edge : DEFAULT_EDGES) {
-			edgeList.add(new Edge<Long, Double>((Long) edge[0], (Long) edge[1], (Double) edge[2]));
-		}
-		return env.fromCollection(edgeList);
-	}
-
-	private SingleSourceShortestPathsData() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java
deleted file mode 100644
index 5b2cc3d..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/utils/TriangleCountData.java
+++ /dev/null
@@ -1,56 +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.example.utils;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.types.NullValue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the Triangle Count test program.
- * If no parameters are given to the program, the default data sets are used.
- */
-public class TriangleCountData {
-
-	public static final String EDGES = "1	2\n"+"1	3\n"+"2	3\n"+"2	6\n"+"3	4\n"+"3	5\n"+"3	6\n"+"4	5\n"+"6	7\n";
-
-	public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvironment env) {
-
-		List<Edge<Long, NullValue>> edges = new ArrayList<Edge<Long, NullValue>>();
-		edges.add(new Edge<Long, NullValue>(1L, 2L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(1L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 3L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(2L, 6L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 4L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(3L, 6L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(4L, 5L, NullValue.getInstance()));
-		edges.add(new Edge<Long, NullValue>(6L, 7L, NullValue.getInstance()));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final String RESULTED_NUMBER_OF_TRIANGLES = "3";
-
-	private TriangleCountData () {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
deleted file mode 100755
index 5a8e97a..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
+++ /dev/null
@@ -1,155 +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.gsa;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * The base class for the third and last step of a {@link GatherSumApplyIteration}.
- *
- * @param <K> the vertex ID type
- * @param <VV> the vertex value type
- * @param <M> the input type (produced by the Sum phase)
- */
-@SuppressWarnings("serial")
-public abstract class ApplyFunction<K, VV, M> implements Serializable {
-
-	// --------------------------------------------------------------------------------------------
-	//  Attribute that allows access to the total number of vertices inside an iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private long numberOfVertices = -1L;
-
-	/**
-	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
-	 * option has been set; -1 otherwise.
-	 */
-	public long getNumberOfVertices() {
-		return numberOfVertices;
-	}
-
-	void setNumberOfVertices(long numberOfVertices) {
-		this.numberOfVertices = numberOfVertices;
-	}
-
-	//---------------------------------------------------------------------------------------------
-
-	/**
-	 * This method is invoked once per superstep, after the {@link SumFunction} 
-	 * in a {@link GatherSumApplyIteration}.
-	 * It updates the Vertex values.
-	 * 
-	 * @param newValue the value computed during the current superstep.
-	 * @param currentValue the current Vertex value.
-	 */
-	public abstract void apply(M newValue, VV currentValue);
-
-	/**
-	 * Sets the result for the apply function
-	 *
-	 * @param result the result of the apply phase
-	 */
-	public void setResult(VV result) {
-		outVal.f1 = result;
-		out.collect(outVal);
-	}
-
-	/**
-	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
-	 */
-	public void preSuperstep() {}
-
-	/**
-	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
-	 */
-	public void postSuperstep() {}
-
-	/**
-	 * Gets the number of the superstep, starting at <tt>1</tt>.
-	 *
-	 * @return The number of the current superstep.
-	 */
-	public int getSuperstepNumber() {
-		return this.runtimeContext.getSuperstepNumber();
-	}
-
-	/**
-	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
-	 * all aggregates globally once per superstep and makes them available in the next superstep.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
-	 */
-	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-		return this.runtimeContext.<T>getIterationAggregator(name);
-	}
-
-	/**
-	 * Get the aggregated value that an aggregator computed in the previous iteration.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregated value of the previous iteration.
-	 */
-	public <T extends Value> T getPreviousIterationAggregate(String name) {
-		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
-	}
-
-	/**
-	 * Gets the broadcast data set registered under the given name. Broadcast data sets
-	 * are available on all parallel instances of a function.
-	 *
-	 * @param name The name under which the broadcast set is registered.
-	 * @return The broadcast data set.
-	 */
-	public <T> Collection<T> getBroadcastSet(String name) {
-		return this.runtimeContext.<T>getBroadcastVariable(name);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal methods
-	// --------------------------------------------------------------------------------------------
-
-	private IterationRuntimeContext runtimeContext;
-
-	private Collector<Vertex<K, VV>> out;
-
-	private Vertex<K, VV> outVal;
-
-	public void init(IterationRuntimeContext iterationRuntimeContext) {
-		this.runtimeContext = iterationRuntimeContext;
-	}
-
-	public void setOutput(Vertex<K, VV> vertex, Collector<Vertex<K, VV>> out) {
-		this.out = out;
-		this.outVal = vertex;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
deleted file mode 100644
index 8d24f16..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GSAConfiguration.java
+++ /dev/null
@@ -1,133 +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.gsa;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.IterationConfiguration;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A GSAConfiguration object can be used to set the iteration name and
- * degree of parallelism, to register aggregators and use broadcast sets in
- * the {@link org.apache.flink.graph.gsa.GatherFunction}, {@link org.apache.flink.graph.gsa.SumFunction} as well as
- * {@link org.apache.flink.graph.gsa.ApplyFunction}.
- *
- * The GSAConfiguration object is passed as an argument to
- * {@link org.apache.flink.graph.Graph#runGatherSumApplyIteration(org.apache.flink.graph.gsa.GatherFunction,
- * org.apache.flink.graph.gsa.SumFunction, org.apache.flink.graph.gsa.ApplyFunction, int)}
- */
-public class GSAConfiguration extends IterationConfiguration {
-
-	/** the broadcast variables for the gather function **/
-	private List<Tuple2<String, DataSet<?>>> bcVarsGather = new ArrayList<Tuple2<String,DataSet<?>>>();
-
-	/** the broadcast variables for the sum function **/
-	private List<Tuple2<String, DataSet<?>>> bcVarsSum = new ArrayList<Tuple2<String,DataSet<?>>>();
-
-	/** the broadcast variables for the apply function **/
-	private List<Tuple2<String, DataSet<?>>> bcVarsApply = new ArrayList<Tuple2<String,DataSet<?>>>();
-
-	private EdgeDirection direction = EdgeDirection.OUT;
-
-	public GSAConfiguration() {}
-
-	/**
-	 * Adds a data set as a broadcast set to the gather function.
-	 *
-	 * @param name The name under which the broadcast data is available in the gather function.
-	 * @param data The data set to be broadcasted.
-	 */
-	public void addBroadcastSetForGatherFunction(String name, DataSet<?> data) {
-		this.bcVarsGather.add(new Tuple2<String, DataSet<?>>(name, data));
-	}
-
-	/**
-	 * Adds a data set as a broadcast set to the sum function.
-	 *
-	 * @param name The name under which the broadcast data is available in the sum function.
-	 * @param data The data set to be broadcasted.
-	 */
-	public void addBroadcastSetForSumFunction(String name, DataSet<?> data) {
-		this.bcVarsSum.add(new Tuple2<String, DataSet<?>>(name, data));
-	}
-
-	/**
-	 * Adds a data set as a broadcast set to the apply function.
-	 *
-	 * @param name The name under which the broadcast data is available in the apply function.
-	 * @param data The data set to be broadcasted.
-	 */
-	public void addBroadcastSetForApplyFunction(String name, DataSet<?> data) {
-		this.bcVarsApply.add(new Tuple2<String, DataSet<?>>(name, data));
-	}
-
-	/**
-	 * Get the broadcast variables of the GatherFunction.
-	 *
-	 * @return a List of Tuple2, where the first field is the broadcast variable name
-	 * and the second field is the broadcast DataSet.
-	 */
-	public List<Tuple2<String, DataSet<?>>> getGatherBcastVars() {
-		return this.bcVarsGather;
-	}
-
-	/**
-	 * Get the broadcast variables of the SumFunction.
-	 *
-	 * @return a List of Tuple2, where the first field is the broadcast variable name
-	 * and the second field is the broadcast DataSet.
-	 */
-	public List<Tuple2<String, DataSet<?>>> getSumBcastVars() {
-		return this.bcVarsSum;
-	}
-
-	/**
-	 * Get the broadcast variables of the ApplyFunction.
-	 *
-	 * @return a List of Tuple2, where the first field is the broadcast variable name
-	 * and the second field is the broadcast DataSet.
-	 */
-	public List<Tuple2<String, DataSet<?>>> getApplyBcastVars() {
-		return this.bcVarsApply;
-	}
-
-	/**
-	 * Gets the direction from which the neighbors are to be selected
-	 * By default the neighbors who are target of the edges are selected
-	 *
-	 * @return an EdgeDirection, which can be either IN, OUT or ALL.
-	 */
-	public EdgeDirection getDirection() {
-		return direction;
-	}
-
-	/**
-	 * Sets the direction in which neighbors are to be selected
-	 * By default the neighbors who are target of the edges are selected
-	 *
-	 * @param direction - IN, OUT or ALL
-	 */
-	public void setDirection(EdgeDirection direction) {
-		this.direction = direction;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
deleted file mode 100755
index 563b20e..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
+++ /dev/null
@@ -1,134 +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.gsa;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.types.Value;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * The base class for the first step of a {@link GatherSumApplyIteration}.
- * 
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- * @param <M> the output type 
- */
-@SuppressWarnings("serial")
-public abstract class GatherFunction<VV, EV, M> implements Serializable {
-
-	// --------------------------------------------------------------------------------------------
-	//  Attribute that allows access to the total number of vertices inside an iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private long numberOfVertices = -1L;
-
-	/**
-	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
-	 * option has been set; -1 otherwise.
-	 */
-	public long getNumberOfVertices() {
-		return numberOfVertices;
-	}
-
-	void setNumberOfVertices(long numberOfVertices) {
-		this.numberOfVertices = numberOfVertices;
-	}
-
-	//---------------------------------------------------------------------------------------------
-
-	/**
-	 * This method is invoked once per superstep, for each {@link Neighbor} of each Vertex 
-	 * in the beginning of each superstep in a {@link GatherSumApplyIteration}.
-	 * It needs to produce a partial value, which will be combined with other partial value
-	 * in the next phase of the iteration.
-	 *  
-	 * @param neighbor the input Neighbor. It provides access to the source Vertex and the Edge objects.
-	 * @return a partial result to be combined in the Sum phase.
-	 */
-	public abstract M gather(Neighbor<VV, EV> neighbor);
-
-	/**
-	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
-	 */
-	public void preSuperstep() {}
-
-	/**
-	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
-	 */
-	public void postSuperstep() {}
-
-	/**
-	 * Gets the number of the superstep, starting at <tt>1</tt>.
-	 *
-	 * @return The number of the current superstep.
-	 */
-	public int getSuperstepNumber() {
-		return this.runtimeContext.getSuperstepNumber();
-	}
-
-	/**
-	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
-	 * all aggregates globally once per superstep and makes them available in the next superstep.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
-	 */
-	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-		return this.runtimeContext.<T>getIterationAggregator(name);
-	}
-
-	/**
-	 * Get the aggregated value that an aggregator computed in the previous iteration.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregated value of the previous iteration.
-	 */
-	public <T extends Value> T getPreviousIterationAggregate(String name) {
-		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
-	}
-
-	/**
-	 * Gets the broadcast data set registered under the given name. Broadcast data sets
-	 * are available on all parallel instances of a function.
-	 *
-	 * @param name The name under which the broadcast set is registered.
-	 * @return The broadcast data set.
-	 */
-	public <T> Collection<T> getBroadcastSet(String name) {
-		return this.runtimeContext.<T>getBroadcastVariable(name);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal methods
-	// --------------------------------------------------------------------------------------------
-
-	private IterationRuntimeContext runtimeContext;
-
-	public void init(IterationRuntimeContext iterationRuntimeContext) {
-		this.runtimeContext = iterationRuntimeContext;
-	}
-}


[11/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
deleted file mode 100644
index 3dc90fc..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithEdgesITCase.scala
+++ /dev/null
@@ -1,151 +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.scala.test.operations
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.Edge
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.scala.utils.EdgeToTuple3Map
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class JoinWithEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesInputDataset {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new
-        EdgeToTuple3Map[Long, Long]), new AddValuesMapper)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
-      "90\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesInputDatasetSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdges(graph.getEdges.map(new
-        EdgeToTuple3Map[Long, Long]), (originalValue: Long, tupleValue: Long) =>
-      originalValue + tupleValue)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,46\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
-      "90\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesOnSource {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges
-      .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) =>
-      originalValue + tupleValue)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," +
-      "90\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesOnSourceSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnSource[Long](graph.getEdges
-      .map(new ProjectSourceAndValueMapper), (originalValue: Long, tupleValue: Long) =>
-      originalValue + tupleValue)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,25\n" + "2,3,46\n" + "3,4,68\n" + "3,5,69\n" + "4,5," +
-      "90\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesOnTarget {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges
-      .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) =>
-      originalValue + tupleValue)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
-      "80\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithEdgesOnTargetSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithEdgesOnTarget[Long](graph.getEdges
-      .map(new ProjectTargetAndValueMapper), (originalValue: Long, tupleValue: Long) =>
-      originalValue + tupleValue)
-    val res = result.getEdges.collect().toList
-    expectedResult = "1,2,24\n" + "1,3,26\n" + "2,3,36\n" + "3,4,68\n" + "3,5,70\n" + "4,5," +
-      "80\n" + "5,1,102\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-
-  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
-    @throws(classOf[Exception])
-    def map(tuple: (Long, Long)): Long = {
-      tuple._1 + tuple._2
-    }
-  }
-
-  final class ProjectSourceAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] {
-    @throws(classOf[Exception])
-    def map(edge: Edge[Long, Long]): (Long, Long) = {
-      (edge.getSource, edge.getValue)
-    }
-  }
-
-  final class ProjectTargetAndValueMapper extends MapFunction[Edge[Long, Long], (Long, Long)] {
-    @throws(classOf[Exception])
-    def map(edge: Edge[Long, Long]): (Long, Long) = {
-      (edge.getTarget, edge.getValue)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
deleted file mode 100644
index 98ee8b6..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/JoinWithVerticesITCase.scala
+++ /dev/null
@@ -1,74 +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.scala.test.operations
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.scala.utils.VertexToTuple2Map
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class JoinWithVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testJoinWithVertexSet {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result: Graph[Long, Long, Long] = graph.joinWithVertices(graph.getVertices.map(new
-        VertexToTuple2Map[Long, Long]), new AddValuesMapper)
-    val res = result.getVertices.collect().toList
-    expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testJoinWithVertexSetSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val tupleSet = graph.getVertices.map(new VertexToTuple2Map[Long, Long])
-    val result: Graph[Long, Long, Long] = graph.joinWithVertices[Long](tupleSet,
-      (originalvalue: Long, tuplevalue: Long) => originalvalue + tuplevalue)
-    val res = result.getVertices.collect().toList
-    expectedResult = "1,2\n" + "2,4\n" + "3,6\n" + "4,8\n" + "5,10\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-
-  final class AddValuesMapper extends MapFunction[(Long, Long), Long] {
-    @throws(classOf[Exception])
-    def map(tuple: (Long, Long)): Long = {
-      tuple._1 + tuple._2
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
deleted file mode 100644
index bdfd569..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapEdgesITCase.scala
+++ /dev/null
@@ -1,81 +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.scala.test.operations
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.Edge
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class MapEdgesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithSameValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.mapEdges(new AddOneMapper).getEdges.collect().toList
-    expectedResult = "1,2,13\n" +
-      "1,3,14\n" + "" +
-      "2,3,24\n" +
-      "3,4,35\n" +
-      "3,5,36\n" +
-      "4,5,46\n" +
-      "5,1,52\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithSameValueSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.mapEdges(edge => edge.getValue + 1)
-      .getEdges.collect().toList
-    expectedResult = "1,2,13\n" +
-      "1,3,14\n" + "" +
-      "2,3,24\n" +
-      "3,4,35\n" +
-      "3,5,36\n" +
-      "4,5,46\n" +
-      "5,1,52\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  final class AddOneMapper extends MapFunction[Edge[Long, Long], Long] {
-    @throws(classOf[Exception])
-    def map(edge: Edge[Long, Long]): Long = {
-      edge.getValue + 1
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
deleted file mode 100644
index 2e51d90..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
+++ /dev/null
@@ -1,76 +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.scala.test.operations
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.Vertex
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class MapVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithSameValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.mapVertices(new AddOneMapper).getVertices.collect().toList
-    expectedResult = "1,2\n" +
-      "2,3\n" +
-      "3,4\n" +
-      "4,5\n" +
-      "5,6\n";
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testWithSameValueSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.mapVertices(vertex => vertex.getValue + 1).getVertices.collect().toList
-    expectedResult = "1,2\n" +
-      "2,3\n" +
-      "3,4\n" +
-      "4,5\n" +
-      "5,6\n";
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  final class AddOneMapper extends MapFunction[Vertex[Long, Long], Long] {
-    @throws(classOf[Exception])
-    def map(vertex: Vertex[Long, Long]): Long = {
-      vertex.getValue + 1
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
deleted file mode 100644
index dcd1deb..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
+++ /dev/null
@@ -1,150 +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.scala.test.operations
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.scala.{EdgesFunction, EdgesFunctionWithVertexValue, Graph}
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.apache.flink.util.Collector
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class ReduceOnEdgesMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode)
-  extends MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testAllNeighborsWithValueGreaterThanFour {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour,
-      EdgeDirection.ALL).collect().toList
-    expectedResult = "(5,1)\n" + "(5,3)\n" + "(5,4)"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-
-  @Test
-  @throws(classOf[Exception])
-  def testAllNeighbors {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.groupReduceOnEdges(new SelectNeighbors, EdgeDirection.ALL)
-    .collect().toList
-    expectedResult = "(1,2)\n" + "(1,3)\n" + "(1,5)\n" + "(2,1)\n" + "(2,3)\n" +
-      "(3,1)\n" + "(3,2)\n" + "(3,4)\n" + "(3,5)\n" + "(4,3)\n" + "(4,5)\n" +
-      "(5,1)\n" + "(5,3)\n" + "(5,4)"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testLowestWeightOutNeighborNoValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new
-        SelectMinWeightNeighborNoValue, EdgeDirection.OUT)
-    val res = verticesWithLowestOutNeighbor.collect().toList
-    expectedResult = "(1,12)\n" + "(2,23)\n" + "(3,34)\n" + "(4,45)\n" + "(5,51)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testLowestWeightInNeighborNoValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new
-        SelectMinWeightNeighborNoValue, EdgeDirection.IN)
-    val res = verticesWithLowestOutNeighbor.collect().toList
-    expectedResult = "(1,51)\n" + "(2,12)\n" + "(3,13)\n" + "(4,34)\n" + "(5,35)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testMaxWeightAllNeighbors {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val verticesWithMaxEdgeWeight: DataSet[(Long, Long)] = graph.reduceOnEdges(new
-        SelectMaxWeightNeighborNoValue, EdgeDirection.ALL)
-    val res = verticesWithMaxEdgeWeight.collect().toList
-    expectedResult = "(1,51)\n" + "(2,23)\n" + "(3,35)\n" + "(4,45)\n" + "(5,51)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  final class SelectNeighborsValueGreaterThanFour extends EdgesFunctionWithVertexValue[Long,
-    Long, Long, (Long, Long)] {
-    @throws(classOf[Exception])
-    override def iterateEdges(v: Vertex[Long, Long], edges: Iterable[Edge[Long, Long]], out:
-    Collector[(Long, Long)]): Unit = {
-      for (edge <- edges) {
-        if (v.getValue > 4) {
-          if (v.getId == edge.getTarget) {
-            out.collect((v.getId, edge.getSource))
-          }
-          else {
-            out.collect((v.getId, edge.getTarget))
-          }
-        }
-      }
-    }
-  }
-
-  final class SelectNeighbors extends EdgesFunction[Long, Long, (Long, Long)] {
-    @throws(classOf[Exception])
-    override def iterateEdges(edges: Iterable[(Long, Edge[Long, Long])], out: Collector[
-      (Long, Long)]) {
-      for (edge <- edges) {
-        if (edge._1.equals(edge._2.getTarget)) {
-          out.collect(new Tuple2[Long, Long](edge._1, edge._2.getSource))
-        }
-        else {
-          out.collect(new Tuple2[Long, Long](edge._1, edge._2.getTarget))
-        }
-      }
-    }
-  }
-
-  final class SelectMinWeightNeighborNoValue extends ReduceEdgesFunction[Long] {
-    override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = {
-      Math.min(firstEdgeValue, secondEdgeValue)
-    }
-  }
-
-  final class SelectMaxWeightNeighborNoValue extends ReduceEdgesFunction[Long] {
-    override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = {
-      Math.max(firstEdgeValue, secondEdgeValue)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
deleted file mode 100644
index aef5493..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
+++ /dev/null
@@ -1,126 +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.scala.test.operations
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.scala.{NeighborsFunctionWithVertexValue, _}
-import org.apache.flink.graph.{Edge, EdgeDirection, ReduceNeighborsFunction, Vertex}
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.apache.flink.util.Collector
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class ReduceOnNeighborMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode)
-  extends MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testSumOfAllNeighborsNoValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.ALL)
-    .collect().toList
-    expectedResult = "(1,10)\n" + "(2,4)\n" + "(3,12)\n" + "(4,8)\n" + "(5,8)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testSumOfOutNeighborsNoValue {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.OUT).collect().toList
-    expectedResult = "(1,5)\n" + "(2,3)\n" + "(3,9)\n" + "(4,5)\n" + "(5,1)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testSumOfAllNeighbors {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result = graph.groupReduceOnNeighbors(new SumAllNeighbors, EdgeDirection.ALL)
-    val res = result.collect().toList
-    expectedResult = "(1,11)\n" + "(2,6)\n" + "(3,15)\n" + "(4,12)\n" + "(5,13)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testSumOfInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val result = graph.groupReduceOnNeighbors(new
-        SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo, EdgeDirection.IN)
-    val res = result.collect().toList
-    expectedResult = "(3,59)\n" + "(3,118)\n" + "(4,204)\n" + "(4,102)\n" + "(5,570)\n" + "(5,285)"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  final class SumNeighbors extends ReduceNeighborsFunction[Long] {
-    override def reduceNeighbors(firstNeighbor: Long, secondNeighbor: Long): Long = {
-      firstNeighbor + secondNeighbor
-    }
-  }
-
-  final class SumAllNeighbors extends NeighborsFunctionWithVertexValue[Long, Long, Long, (Long,
-    Long)] {
-    @throws(classOf[Exception])
-    def iterateNeighbors(vertex: Vertex[Long, Long], neighbors: Iterable[(Edge[Long, Long],
-      Vertex[Long, Long])], out: Collector[(Long, Long)]) {
-      var sum: Long = 0
-      for (neighbor <- neighbors) {
-        sum += neighbor._2.getValue
-      }
-      out.collect((vertex.getId, sum + vertex.getValue))
-    }
-  }
-
-  final class SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo extends
-  NeighborsFunction[Long, Long, Long, (Long, Long)] {
-    @throws(classOf[Exception])
-    def iterateNeighbors(neighbors: Iterable[(Long, Edge[Long, Long], Vertex[Long, Long])],
-                         out: Collector[(Long, Long)]) {
-      var sum: Long = 0
-      var next: (Long, Edge[Long, Long], Vertex[Long, Long]) = null
-      val neighborsIterator: Iterator[(Long, Edge[Long, Long], Vertex[Long, Long])] =
-        neighbors.iterator
-      while (neighborsIterator.hasNext) {
-        next = neighborsIterator.next
-        sum += next._3.getValue * next._2.getValue
-      }
-      if (next._1 > 2) {
-        out.collect(new Tuple2[Long, Long](next._1, sum))
-        out.collect(new Tuple2[Long, Long](next._1, sum * 2))
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/pom.xml b/flink-staging/flink-gelly/pom.xml
deleted file mode 100644
index 9dce170..0000000
--- a/flink-staging/flink-gelly/pom.xml
+++ /dev/null
@@ -1,67 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-staging</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-	
-	<artifactId>flink-gelly</artifactId>
-	<name>flink-gelly</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-optimizer</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-	</dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
deleted file mode 100644
index d84badb..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
+++ /dev/null
@@ -1,75 +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;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-
-/**
- * An Edge represents a link between two {@link Vertex vertices},
- * the source and the target and can carry an attached value.
- * For edges with no value, use {@link org.apache.flink.types.NullValue} as the value type.
- *
- * @param <K> the key type for the sources and target vertices
- * @param <V> the edge value type
- */
-public class Edge<K, V> extends Tuple3<K, K, V>{
-
-	private static final long serialVersionUID = 1L;
-
-	public Edge(){}
-
-	public Edge(K src, K trg, V val) {
-		this.f0 = src;
-		this.f1 = trg;
-		this.f2 = val;
-	}
-
-	/**
-	 * Reverses the direction of this Edge.
-	 * @return a new Edge, where the source is the original Edge's target
-	 * and the target is the original Edge's source.
-	 */
-	public Edge<K, V> reverse() {
-			return new Edge<K, V>(this.f1, this.f0, this.f2);
-	}
-
-	public void setSource(K src) {
-		this.f0 = src;
-	}
-
-	public K getSource() {
-		return this.f0;
-	}
-
-	public void setTarget(K target) {
-		this.f1 = target;
-	}
-
-	public K getTarget() {
-		return f1;
-	}
-
-	public void setValue(V value) {
-		this.f2 = value;
-	}
-
-	public V getValue() {
-		return f2;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
deleted file mode 100644
index 0a055bb..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
+++ /dev/null
@@ -1,35 +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;
-
-/**
- * The EdgeDirection is used to select a node's neighborhood
- * by the {@link Graph#groupReduceOnEdges(EdgesFunction, EdgeDirection)},
- * {@link Graph#groupReduceOnEdges(EdgesFunctionWithVertexValue, EdgeDirection)},
- * {@link Graph#groupReduceOnNeighbors(NeighborsFunction, EdgeDirection)},
- * {@link Graph#groupReduceOnNeighbors(NeighborsFunctionWithVertexValue, EdgeDirection)},
- * {@link Graph#reduceOnEdges(ReduceEdgesFunction, EdgeDirection)} and
- * {@link Graph#reduceOnNeighbors(ReduceNeighborsFunction, EdgeDirection)}
- * methods.
- */
-public enum EdgeDirection {
-	IN,
-	OUT,
-	ALL
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
deleted file mode 100644
index bf1d6a2..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
+++ /dev/null
@@ -1,38 +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;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.Collector;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#groupReduceOnEdges(EdgesFunction, EdgeDirection)} method.
- *
- * @param <K> the vertex key type
- * @param <EV> the edge value type
- * @param <O> the type of the return value
- */
-public interface EdgesFunction<K, EV, O> extends Function, Serializable {
-
-	void iterateEdges(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
deleted file mode 100644
index 0b0ab0e..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
+++ /dev/null
@@ -1,39 +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;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.util.Collector;
-
-/**
- * Interface to be implemented by the function applied to a vertex neighborhood
- * in the {@link Graph#groupReduceOnEdges(EdgesFunctionWithVertexValue, EdgeDirection)}
- * method.
- *
- * @param <K> the vertex key type
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- * @param <O> the type of the return value
- */
-public interface EdgesFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
-
-	void iterateEdges(Vertex<K, VV> v, Iterable<Edge<K, EV>> edges, Collector<O> out) throws Exception;
-}


[05/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
deleted file mode 100644
index bb3a131..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelTranslationTest.java
+++ /dev/null
@@ -1,231 +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.spargel;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.junit.Test;
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.operators.DeltaIterationResultSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.TwoInputUdfOperator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.NullValue;
-
-@SuppressWarnings("serial")
-public class SpargelTranslationTest {
-
-	@Test
-	public void testTranslationPlainEdges() {
-		try {
-			final String ITERATION_NAME = "Test Name";
-			
-			final String AGGREGATOR_NAME = "AggregatorName";
-			
-			final String BC_SET_MESSAGES_NAME = "borat messages";
-			
-			final String BC_SET_UPDATES_NAME = "borat updates";
-			;
-			final int NUM_ITERATIONS = 13;
-			
-			final int ITERATION_parallelism = 77;
-			
-			
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
-			DataSet<Long> bcMessaging = env.fromElements(1L);
-			DataSet<Long> bcUpdate = env.fromElements(1L);
-			
-			DataSet<Vertex<String, Double>> result;
-			
-			// ------------ construct the test program ------------------
-			{
-				
-				DataSet<Tuple2<String, Double>> initialVertices = env.fromElements(new Tuple2<String, Double>("abc", 3.44));
-
-				DataSet<Tuple2<String, String>> edges = env.fromElements(new Tuple2<String, String>("a", "c"));
-
-				Graph<String, Double, NullValue> graph = Graph.fromTupleDataSet(initialVertices,
-						edges.map(new MapFunction<Tuple2<String,String>, Tuple3<String, String, NullValue>>() {
-
-							public Tuple3<String, String, NullValue> map(
-									Tuple2<String, String> edge) {
-								return new Tuple3<String, String, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
-							}
-						}), env);
-
-				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-				parameters.addBroadcastSetForMessagingFunction(BC_SET_MESSAGES_NAME, bcMessaging);
-				parameters.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcUpdate);
-				parameters.setName(ITERATION_NAME);
-				parameters.setParallelism(ITERATION_parallelism);
-				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
-
-				result = graph.runVertexCentricIteration(new UpdateFunction(), new MessageFunctionNoEdgeValue(),
-						NUM_ITERATIONS, parameters).getVertices();
-
-				result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
-			}
-			
-			
-			// ------------- validate the java program ----------------
-			
-			assertTrue(result instanceof DeltaIterationResultSet);
-			
-			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
-			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
-			
-			// check the basic iteration properties
-			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
-			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
-			assertEquals(ITERATION_parallelism, iteration.getParallelism());
-			assertEquals(ITERATION_NAME, iteration.getName());
-			
-			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
-			
-			// validate that the semantic properties are set as they should
-			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
-			
-			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
-			
-			// validate that the broadcast sets are forwarded
-			assertEquals(bcUpdate, solutionSetJoin.getBroadcastSets().get(BC_SET_UPDATES_NAME));
-			assertEquals(bcMessaging, edgesJoin.getBroadcastSets().get(BC_SET_MESSAGES_NAME));
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testTranslationPlainEdgesWithForkedBroadcastVariable() {
-		try {
-			final String ITERATION_NAME = "Test Name";
-			
-			final String AGGREGATOR_NAME = "AggregatorName";
-			
-			final String BC_SET_MESSAGES_NAME = "borat messages";
-			
-			final String BC_SET_UPDATES_NAME = "borat updates";
-			;
-			final int NUM_ITERATIONS = 13;
-			
-			final int ITERATION_parallelism = 77;
-			
-			
-			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
-			DataSet<Long> bcVar = env.fromElements(1L);
-			
-			DataSet<Vertex<String, Double>> result;
-			
-			// ------------ construct the test program ------------------
-			{
-
-				DataSet<Tuple2<String, Double>> initialVertices = env.fromElements(new Tuple2<String, Double>("abc", 3.44));
-
-				DataSet<Tuple2<String, String>> edges = env.fromElements(new Tuple2<String, String>("a", "c"));
-
-				Graph<String, Double, NullValue> graph = Graph.fromTupleDataSet(initialVertices,
-						edges.map(new MapFunction<Tuple2<String,String>, Tuple3<String, String, NullValue>>() {
-
-							public Tuple3<String, String, NullValue> map(
-									Tuple2<String, String> edge) {
-								return new Tuple3<String, String, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
-							}
-						}), env);
-
-				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-				parameters.addBroadcastSetForMessagingFunction(BC_SET_MESSAGES_NAME, bcVar);
-				parameters.addBroadcastSetForUpdateFunction(BC_SET_UPDATES_NAME, bcVar);
-				parameters.setName(ITERATION_NAME);
-				parameters.setParallelism(ITERATION_parallelism);
-				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
-				
-				result = graph.runVertexCentricIteration(new UpdateFunction(), new MessageFunctionNoEdgeValue(),
-						NUM_ITERATIONS, parameters).getVertices();
-
-				result.output(new DiscardingOutputFormat<Vertex<String, Double>>());
-			}
-			
-			
-			// ------------- validate the java program ----------------
-			
-			assertTrue(result instanceof DeltaIterationResultSet);
-			
-			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
-			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
-			
-			// check the basic iteration properties
-			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
-			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
-			assertEquals(ITERATION_parallelism, iteration.getParallelism());
-			assertEquals(ITERATION_NAME, iteration.getName());
-			
-			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
-			
-			// validate that the semantic properties are set as they should
-			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
-			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
-			
-			TwoInputUdfOperator<?, ?, ?, ?> edgesJoin = (TwoInputUdfOperator<?, ?, ?, ?>) solutionSetJoin.getInput1();
-			
-			// validate that the broadcast sets are forwarded
-			assertEquals(bcVar, solutionSetJoin.getBroadcastSets().get(BC_SET_UPDATES_NAME));
-			assertEquals(bcVar, edgesJoin.getBroadcastSets().get(BC_SET_MESSAGES_NAME));
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public static class UpdateFunction extends VertexUpdateFunction<String, Double, Long> {
-
-		@Override
-		public void updateVertex(Vertex<String, Double> vertex, MessageIterator<Long> inMessages) {}
-	}
-	
-	public static class MessageFunctionNoEdgeValue extends MessagingFunction<String, Double, Long, NullValue> {
-
-		@Override
-		public void sendMessages(Vertex<String, Double> vertex) {}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
deleted file mode 100644
index 3fbd0bc..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/CollectionModeSuperstepITCase.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.graph.test;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.graph.utils.VertexToTuple2Map;
-import org.junit.Assert;
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class CollectionModeSuperstepITCase {
-
-	/**
-	 * Dummy iteration to test that the supersteps are correctly incremented
-	 * and can be retrieved from inside the updated and messaging functions.
-	 * All vertices start with value 1 and increase their value by 1
-	 * in each iteration. 
-	 */
-	@Test
-	public void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
-				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
-		
-		Graph<Long, Long, Long> result = graph.runVertexCentricIteration(
-				new UpdateFunction(), new MessageFunction(), 10);
-
-		result.getVertices().map(
-				new VertexToTuple2Map<Long, Long>()).output(
-						new DiscardingOutputFormat<Tuple2<Long, Long>>());
-		env.execute();
-	}
-	
-	public static final class UpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			long superstep = getSuperstepNumber();
-			Assert.assertEquals(true, vertex.getValue() == superstep);
-			setNewVertexValue(vertex.getValue() + 1);
-		}
-	}
-	
-	public static final class MessageFunction extends MessagingFunction<Long, Long, Long, Long> {
-		@Override
-		public void sendMessages(Vertex<Long, Long> vertex) {
-			long superstep = getSuperstepNumber();
-			Assert.assertEquals(true, vertex.getValue() == superstep);
-			//send message to keep vertices active
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-
-	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
-
-		public Long map(Vertex<Long, Long> value) {
-			return 1l;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
deleted file mode 100644
index 67d32a8..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyConfigurationITCase.java
+++ /dev/null
@@ -1,404 +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;
-
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-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.Tuple1;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.gsa.ApplyFunction;
-import org.apache.flink.graph.gsa.GSAConfiguration;
-import org.apache.flink.graph.gsa.GatherFunction;
-import org.apache.flink.graph.gsa.GatherSumApplyIteration;
-import org.apache.flink.graph.gsa.Neighbor;
-import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.LongValue;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.HashSet;
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class GatherSumApplyConfigurationITCase extends MultipleProgramsTestBase {
-
-	public GatherSumApplyConfigurationITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private String expectedResult;
-
-	@Test
-	public void testRunWithConfiguration() throws Exception {
-		/*
-		 * Test Graph's runGatherSumApplyIteration when configuration parameters are provided
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
-
-		// create the configuration object
-		GSAConfiguration parameters = new GSAConfiguration();
-
-		parameters.addBroadcastSetForGatherFunction("gatherBcastSet", env.fromElements(1, 2, 3));
-		parameters.addBroadcastSetForSumFunction("sumBcastSet", env.fromElements(4, 5, 6));
-		parameters.addBroadcastSetForApplyFunction("applyBcastSet", env.fromElements(7, 8, 9));
-		parameters.registerAggregator("superstepAggregator", new LongSumAggregator());
-		parameters.setOptNumVertices(true);
-
-		Graph<Long, Long, Long> res = graph.runGatherSumApplyIteration(new Gather(), new Sum(),
-				new Apply(), 10, parameters);
-
-        DataSet<Vertex<Long, Long>> data = res.getVertices();
-        List<Vertex<Long, Long>> result= data.collect();
-
-		expectedResult = "1,11\n" +
-				"2,11\n" +
-				"3,11\n" +
-				"4,11\n" +
-				"5,11";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationConfiguration() throws Exception {
-
-		/*
-		 * Test name, parallelism and solutionSetUnmanaged parameters
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		GatherSumApplyIteration<Long, Long, Long, Long> iteration = GatherSumApplyIteration
-				.withEdges(TestGraphUtils.getLongLongEdgeData(env), new DummyGather(),
-						new DummySum(), new DummyApply(), 10);
-
-		GSAConfiguration parameters = new GSAConfiguration();
-		parameters.setName("gelly iteration");
-		parameters.setParallelism(2);
-		parameters.setSolutionSetUnmanagedMemory(true);
-
-		iteration.configure(parameters);
-
-		Assert.assertEquals("gelly iteration", iteration.getIterationConfiguration().getName(""));
-		Assert.assertEquals(2, iteration.getIterationConfiguration().getParallelism());
-		Assert.assertEquals(true, iteration.getIterationConfiguration().isSolutionSetUnmanagedMemory());
-
-		DataSet<Vertex<Long, Long>> data = TestGraphUtils.getLongLongVertexData(env).runOperation(iteration);
-        List<Vertex<Long, Long>> result= data.collect();
-        
-		expectedResult = "1,11\n" +
-				"2,12\n" +
-				"3,13\n" +
-				"4,14\n" +
-				"5,15";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationDefaultDirection() throws Exception {
-
-		/*
-		 * Test that if no direction parameter is given, the iteration works as before
-		 * (i.e. it gathers information from the IN edges and neighbors and the information is calculated for an OUT edge
-		 * Default direction parameter is OUT for the GatherSumApplyIterations)
-		 * When data is gathered from the IN edges the Gather Sum and Apply functions
-		 * set the set of vertices which have path to a vertex as the value of that vertex
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
-
-		edges.remove(0);
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
-				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
-				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4)
-				.getVertices();
-
-		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
-
-		expectedResult = "1,[1, 2, 3, 4, 5]\n"
-						+"2,[2]\n"
-						+"3,[1, 2, 3, 4, 5]\n"
-						+"4,[1, 2, 3, 4, 5]\n"
-						+"5,[1, 2, 3, 4, 5]\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationDirectionIN() throws Exception {
-
-		/*
-		 * Test that if the direction parameter IN is given, the iteration works as expected
-		 * (i.e. it gathers information from the OUT edges and neighbors and the information is calculated for an IN edge
-		 * When data is gathered from the OUT edges the Gather Sum and Apply functions
-		 * set the set of vertices which have path from a vertex as the value of that vertex
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		GSAConfiguration parameters = new GSAConfiguration();
-
-		parameters.setDirection(EdgeDirection.IN);
-
-		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
-
-		edges.remove(0);
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
-				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
-				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4,
-																								parameters)
-				.getVertices();
-		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
-
-		expectedResult = "1,[1, 3, 4, 5]\n"
-				+"2,[1, 2, 3, 4, 5]\n"
-				+"3,[1, 3, 4, 5]\n"
-				+"4,[1, 3, 4, 5]\n"
-				+"5,[1, 3, 4, 5]\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationDirectionALL() throws Exception {
-
-		/*
-		 * Test that if the direction parameter OUT is given, the iteration works as expected
-		 * (i.e. it gathers information from both IN and OUT edges and neighbors
-		 * When data is gathered from the ALL edges the Gather Sum and Apply functions
-		 * set the set of vertices which are connected to a Vertex through some path as value of that vertex
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		GSAConfiguration parameters = new GSAConfiguration();
-		parameters.setDirection(EdgeDirection.ALL);
-
-		List<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdges();
-
-		edges.remove(0);
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), edges, env)
-				.mapVertices(new GatherSumApplyConfigurationITCase.InitialiseHashSetMapper());
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph.runGatherSumApplyIteration(
-				new GetReachableVertices(), new FindAllReachableVertices(), new UpdateReachableVertices(), 4,
-				parameters)
-				.getVertices();
-
-		List<Vertex<Long, HashSet<Long>>> result = resultedVertices.collect();
-
-		expectedResult = "1,[1, 2, 3, 4, 5]\n"
-				+"2,[1, 2, 3, 4, 5]\n"
-				+"3,[1, 2, 3, 4, 5]\n"
-				+"4,[1, 2, 3, 4, 5]\n"
-				+"5,[1, 2, 3, 4, 5]\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class Gather extends GatherFunction<Long, Long, Long> {
-
-		@Override
-		public void preSuperstep() {
-
-			// test bcast variable
-			@SuppressWarnings("unchecked")
-			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("gatherBcastSet");
-			Assert.assertEquals(1, bcastSet.get(0));
-			Assert.assertEquals(2, bcastSet.get(1));
-			Assert.assertEquals(3, bcastSet.get(2));
-
-			// test aggregator
-			if (getSuperstepNumber() == 2) {
-				long aggrValue = ((LongValue)getPreviousIterationAggregate("superstepAggregator")).getValue();
-
-				Assert.assertEquals(7, aggrValue);
-			}
-
-			// test number of vertices
-			Assert.assertEquals(5, getNumberOfVertices());
-		}
-
-		public Long gather(Neighbor<Long, Long> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class Sum extends SumFunction<Long, Long, Long> {
-
-		LongSumAggregator aggregator = new LongSumAggregator();
-
-		@Override
-		public void preSuperstep() {
-
-			// test bcast variable
-			@SuppressWarnings("unchecked")
-			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("sumBcastSet");
-			Assert.assertEquals(4, bcastSet.get(0));
-			Assert.assertEquals(5, bcastSet.get(1));
-			Assert.assertEquals(6, bcastSet.get(2));
-
-			// test aggregator
-			aggregator = getIterationAggregator("superstepAggregator");
-
-			// test number of vertices
-			Assert.assertEquals(5, getNumberOfVertices());
-		}
-
-		public Long sum(Long newValue, Long currentValue) {
-			long superstep = getSuperstepNumber();
-			aggregator.aggregate(superstep);
-			return 0l;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class Apply extends ApplyFunction<Long, Long, Long> {
-
-		LongSumAggregator aggregator = new LongSumAggregator();
-
-		@Override
-		public void preSuperstep() {
-
-			// test bcast variable
-			@SuppressWarnings("unchecked")
-			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("applyBcastSet");
-			Assert.assertEquals(7, bcastSet.get(0));
-			Assert.assertEquals(8, bcastSet.get(1));
-			Assert.assertEquals(9, bcastSet.get(2));
-
-			// test aggregator
-			aggregator = getIterationAggregator("superstepAggregator");
-
-			// test number of vertices
-			Assert.assertEquals(5, getNumberOfVertices());
-		}
-
-		public void apply(Long summedValue, Long origValue) {
-			long superstep = getSuperstepNumber();
-			aggregator.aggregate(superstep);
-			setResult(origValue + 1);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class DummyGather extends GatherFunction<Long, Long, Long> {
-
-		@Override
-		public void preSuperstep() {
-			// test number of vertices
-			// when the numVertices option is not set, -1 is returned
-			Assert.assertEquals(-1, getNumberOfVertices());
-		}
-
-		public Long gather(Neighbor<Long, Long> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class DummySum extends SumFunction<Long, Long, Long> {
-
-		public Long sum(Long newValue, Long currentValue) {
-			return 0l;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class DummyApply extends ApplyFunction<Long, Long, Long> {
-
-		public void apply(Long summedValue, Long origValue) {
-			setResult(origValue + 1);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
-
-		public Long map(Vertex<Long, Long> value) {
-			return 1l;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class InitialiseHashSetMapper implements MapFunction<Vertex<Long, Long>, HashSet<Long>> {
-
-		@Override
-		public HashSet<Long> map(Vertex<Long, Long> value) throws Exception {
-			HashSet<Long> h = new HashSet<Long>();
-			h.add(value.getId());
-			return h;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class GetReachableVertices extends GatherFunction<HashSet<Long>, Long, HashSet<Long>> {
-
-		@Override
-		public HashSet<Long> gather(Neighbor<HashSet<Long>, Long> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class FindAllReachableVertices extends SumFunction<HashSet<Long>, Long, HashSet<Long>> {
-		@Override
-		public HashSet<Long> sum(HashSet<Long> newSet, HashSet<Long> currentSet) {
-			HashSet<Long> set = currentSet;
-			for(Long l : newSet) {
-				set.add(l);
-			}
-			return set;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class UpdateReachableVertices extends ApplyFunction<Long, HashSet<Long>, HashSet<Long>> {
-
-		@Override
-		public void apply(HashSet<Long> newValue, HashSet<Long> currentValue) {
-			newValue.addAll(currentValue);
-			if(newValue.size()>currentValue.size()) {
-				setResult(newValue);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
deleted file mode 100755
index 0213f02..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/GatherSumApplyITCase.java
+++ /dev/null
@@ -1,106 +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;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData;
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
-import org.apache.flink.graph.library.GSAConnectedComponents;
-import org.apache.flink.graph.library.GSASingleSourceShortestPaths;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.NullValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class GatherSumApplyITCase extends MultipleProgramsTestBase {
-
-	public GatherSumApplyITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	private String expectedResult;
-
-	// --------------------------------------------------------------------------------------------
-	//  Connected Components Test
-	// --------------------------------------------------------------------------------------------
-
-	@Test
-	public void testConnectedComponents() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
-				ConnectedComponentsDefaultData.getDefaultEdgeDataSet(env),
-				new InitMapperCC(), env);
-
-        List<Vertex<Long, Long>> result = inputGraph.run(
-        		new GSAConnectedComponents<Long, NullValue>(16)).collect();
-
-		expectedResult = "1,1\n" +
-				"2,1\n" +
-				"3,1\n" +
-				"4,1\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Single Source Shortest Path Test
-	// --------------------------------------------------------------------------------------------
-
-	@Test
-	public void testSingleSourceShortestPaths() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
-				SingleSourceShortestPathsData.getDefaultEdgeDataSet(env),
-				new InitMapperSSSP(), env);
-
-        List<Vertex<Long, Double>> result = inputGraph.run(
-        		new GSASingleSourceShortestPaths<Long>(1l, 16)).collect();
-
-		expectedResult = "1,0.0\n" +
-				"2,12.0\n" +
-				"3,13.0\n" +
-				"4,47.0\n" +
-				"5,48.0\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitMapperCC implements MapFunction<Long, Long> {
-		public Long map(Long value) {
-			return value;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitMapperSSSP implements MapFunction<Long, Double> {
-		public Double map(Long value) {
-			return 0.0;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
deleted file mode 100644
index 294926f..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/TestGraphUtils.java
+++ /dev/null
@@ -1,417 +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;
-
-import java.io.PrintStream;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-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.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Vertex;
-
-public class TestGraphUtils {
-
-	public static final DataSet<Vertex<Long, Long>> getLongLongVertexData(
-			ExecutionEnvironment env) {
-
-		return env.fromCollection(getLongLongVertices());
-	}
-	
-	public static final DataSet<Edge<Long, Long>> getLongLongEdgeData(
-			ExecutionEnvironment env) {
-
-		return env.fromCollection(getLongLongEdges());
-	}
-
-	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidSrcData(
-			ExecutionEnvironment env) {
-		List<Edge<Long, Long>> edges = getLongLongEdges();
-
-		edges.remove(1);
-		edges.add(new Edge<Long, Long>(13L, 3L, 13L));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidTrgData(
-			ExecutionEnvironment env) {
-		List<Edge<Long, Long>> edges =  getLongLongEdges();
-
-		edges.remove(0);
-		edges.add(new Edge<Long, Long>(3L, 13L, 13L));
-
-		return env.fromCollection(edges);
-	}
-
-	public static final DataSet<Edge<Long, Long>> getLongLongEdgeInvalidSrcTrgData(
-			ExecutionEnvironment env) {
-		List<Edge<Long, Long>> edges = getLongLongEdges();
-		edges.remove(0);
-		edges.remove(1);
-		edges.remove(2);
-		edges.add(new Edge<Long, Long>(13L, 3L, 13L));
-		edges.add(new Edge<Long, Long>(1L, 12L, 12L));
-		edges.add(new Edge<Long, Long>(13L, 33L, 13L));
-		return env.fromCollection(edges);
-	}
-
-	public static final DataSet<Edge<String, Long>> getStringLongEdgeData(
-			ExecutionEnvironment env) {
-		List<Edge<String, Long>> edges = new ArrayList<Edge<String, Long>>();
-		edges.add(new Edge<String, Long>("1", "2", 12L));
-		edges.add(new Edge<String, Long>("1", "3", 13L));
-		edges.add(new Edge<String, Long>("2", "3", 23L));
-		edges.add(new Edge<String, Long>("3", "4", 34L));
-		edges.add(new Edge<String, Long>("3", "5", 35L));
-		edges.add(new Edge<String, Long>("4", "5", 45L));
-		edges.add(new Edge<String, Long>("5", "1", 51L));
-		return env.fromCollection(edges);
-	}
-
-	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2Data(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
-		tuples.add(new Tuple2<Long, Long>(1L, 10L));
-		tuples.add(new Tuple2<Long, Long>(2L, 20L));
-		tuples.add(new Tuple2<Long, Long>(3L, 30L));
-		tuples.add(new Tuple2<Long, Long>(4L, 40L));
-		tuples.add(new Tuple2<Long, Long>(6L, 60L));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2SourceData(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
-		tuples.add(new Tuple2<Long, Long>(1L, 10L));
-		tuples.add(new Tuple2<Long, Long>(1L, 20L));
-		tuples.add(new Tuple2<Long, Long>(2L, 30L));
-		tuples.add(new Tuple2<Long, Long>(3L, 40L));
-		tuples.add(new Tuple2<Long, Long>(3L, 50L));
-		tuples.add(new Tuple2<Long, Long>(4L, 60L));
-		tuples.add(new Tuple2<Long, Long>(6L, 70L));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple2<Long, Long>> getLongLongTuple2TargetData(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, Long>> tuples = new ArrayList<Tuple2<Long, Long>>();
-		tuples.add(new Tuple2<Long, Long>(2L, 10L));
-		tuples.add(new Tuple2<Long, Long>(3L, 20L));
-		tuples.add(new Tuple2<Long, Long>(3L, 30L));
-		tuples.add(new Tuple2<Long, Long>(4L, 40L));
-		tuples.add(new Tuple2<Long, Long>(6L, 50L));
-		tuples.add(new Tuple2<Long, Long>(6L, 60L));
-		tuples.add(new Tuple2<Long, Long>(1L, 70L));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple3<Long, Long, Long>> getLongLongLongTuple3Data(
-			ExecutionEnvironment env) {
-		List<Tuple3<Long, Long, Long>> tuples = new ArrayList<Tuple3<Long, Long, Long>>();
-		tuples.add(new Tuple3<Long, Long, Long>(1L, 2L, 12L));
-		tuples.add(new Tuple3<Long, Long, Long>(1L, 3L, 13L));
-		tuples.add(new Tuple3<Long, Long, Long>(2L, 3L, 23L));
-		tuples.add(new Tuple3<Long, Long, Long>(3L, 4L, 34L));
-		tuples.add(new Tuple3<Long, Long, Long>(3L, 6L, 36L));
-		tuples.add(new Tuple3<Long, Long, Long>(4L, 6L, 46L));
-		tuples.add(new Tuple3<Long, Long, Long>(6L, 1L, 61L));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2Data(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
-				DummyCustomParameterizedType<Float>>>();
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
-				new DummyCustomParameterizedType<Float>(10, 10f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
-				new DummyCustomParameterizedType<Float>(20, 20f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
-				new DummyCustomParameterizedType<Float>(30, 30f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(4L,
-				new DummyCustomParameterizedType<Float>(40, 40f)));
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2SourceData(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
-				DummyCustomParameterizedType<Float>>>();
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
-				new DummyCustomParameterizedType<Float>(10, 10f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(1L,
-				new DummyCustomParameterizedType<Float>(20, 20f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
-				new DummyCustomParameterizedType<Float>(30, 30f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
-				new DummyCustomParameterizedType<Float>(40, 40f)));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple2<Long, DummyCustomParameterizedType<Float>>> getLongCustomTuple2TargetData(
-			ExecutionEnvironment env) {
-		List<Tuple2<Long, DummyCustomParameterizedType<Float>>> tuples = new ArrayList<Tuple2<Long,
-				DummyCustomParameterizedType<Float>>>();
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(2L,
-				new DummyCustomParameterizedType<Float>(10, 10f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
-				new DummyCustomParameterizedType<Float>(20, 20f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(3L,
-				new DummyCustomParameterizedType<Float>(30, 30f)));
-		tuples.add(new Tuple2<Long, DummyCustomParameterizedType<Float>>(4L,
-				new DummyCustomParameterizedType<Float>(40, 40f)));
-
-		return env.fromCollection(tuples);
-	}
-
-	public static final DataSet<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>> getLongLongCustomTuple3Data(
-			ExecutionEnvironment env) {
-		List<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>> tuples = 
-				new ArrayList<Tuple3<Long, Long, DummyCustomParameterizedType<Float>>>();
-		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(1L, 2L,
-				new DummyCustomParameterizedType<Float>(10, 10f)));
-		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(1L, 3L,
-				new DummyCustomParameterizedType<Float>(20, 20f)));
-		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(2L, 3L,
-				new DummyCustomParameterizedType<Float>(30, 30f)));
-		tuples.add(new Tuple3<Long, Long, DummyCustomParameterizedType<Float>>(3L, 4L,
-				new DummyCustomParameterizedType<Float>(40, 40f)));
-
-		return env.fromCollection(tuples);
-	}
-
-	/**
-	 * A graph with invalid vertex ids
-	 */
-	public static final DataSet<Vertex<Long, Long>> getLongLongInvalidVertexData(
-			ExecutionEnvironment env) {
-		List<Vertex<Long, Long>> vertices = getLongLongVertices();
-
-		vertices.remove(0);
-		vertices.add(new Vertex<Long, Long>(15L, 1L));
-
-		return env.fromCollection(vertices);
-	}
-
-	/**
-	 * A graph that has at least one vertex with no ingoing/outgoing edges
-	 */
-	public static final DataSet<Edge<Long, Long>> getLongLongEdgeDataWithZeroDegree(
-			ExecutionEnvironment env) {
-		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-		edges.add(new Edge<Long, Long>(1L, 2L, 12L));
-		edges.add(new Edge<Long, Long>(1L, 4L, 14L));
-		edges.add(new Edge<Long, Long>(1L, 5L, 15L));
-		edges.add(new Edge<Long, Long>(2L, 3L, 23L));
-		edges.add(new Edge<Long, Long>(3L, 5L, 35L));
-		edges.add(new Edge<Long, Long>(4L, 5L, 45L));
-
-		return env.fromCollection(edges);
-	}
-
-	/**
-	 * Function that produces an ArrayList of vertices
-	 */
-	public static final List<Vertex<Long, Long>> getLongLongVertices() {
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 1L));
-		vertices.add(new Vertex<Long, Long>(2L, 2L));
-		vertices.add(new Vertex<Long, Long>(3L, 3L));
-		vertices.add(new Vertex<Long, Long>(4L, 4L));
-		vertices.add(new Vertex<Long, Long>(5L, 5L));
-
-		return vertices;
-	}
-
-	public static final List<Vertex<Long, Boolean>> getLongBooleanVertices() {
-		List<Vertex<Long, Boolean>> vertices = new ArrayList<Vertex<Long, Boolean>>();
-		vertices.add(new Vertex<Long, Boolean>(1L, true));
-		vertices.add(new Vertex<Long, Boolean>(2L, true));
-		vertices.add(new Vertex<Long, Boolean>(3L, true));
-		vertices.add(new Vertex<Long, Boolean>(4L, true));
-		vertices.add(new Vertex<Long, Boolean>(5L, true));
-
-		return vertices;
-	}
-
-	public static final DataSet<Edge<Long, Long>> getDisconnectedLongLongEdgeData(
-				ExecutionEnvironment env) {
-			List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-			edges.add(new Edge<Long, Long>(1L, 2L, 12L));
-			edges.add(new Edge<Long, Long>(1L, 3L, 13L));
-			edges.add(new Edge<Long, Long>(2L, 3L, 23L));
-			edges.add(new Edge<Long, Long>(4L, 5L, 45L));
-			
-			return env.fromCollection(edges);
-		}
-	
-	/**
-	 * Function that produces an ArrayList of edges
-	 */
-	public static final List<Edge<Long, Long>> getLongLongEdges() {
-		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-		edges.add(new Edge<Long, Long>(1L, 2L, 12L));
-		edges.add(new Edge<Long, Long>(1L, 3L, 13L));
-		edges.add(new Edge<Long, Long>(2L, 3L, 23L));
-		edges.add(new Edge<Long, Long>(3L, 4L, 34L));
-		edges.add(new Edge<Long, Long>(3L, 5L, 35L));
-		edges.add(new Edge<Long, Long>(4L, 5L, 45L));
-		edges.add(new Edge<Long, Long>(5L, 1L, 51L));
-	
-		return edges;
-	}
-
-	public static class DummyCustomType implements Serializable {
-		private static final long serialVersionUID = 1L;
-		
-		private int intField;
-		private boolean booleanField;
-		
-		public DummyCustomType(int intF, boolean boolF) {
-			this.intField = intF;
-			this.booleanField = boolF;
-		}
-		
-		public DummyCustomType() {
-			this.intField = 0;
-			this.booleanField = true;
-		}
-
-		public int getIntField() {
-			return intField;
-		}
-		
-		public void setIntField(int intF) {
-			this.intField = intF;
-		}
-		
-		public boolean getBooleanField() {
-			return booleanField;
-		}
-		
-		@Override
-		public String toString() {
-			return booleanField ? "(T," + intField + ")" : "(F," + intField + ")";
-		}
-	}
-	
-	public static class DummyCustomParameterizedType<T> implements Serializable {
-		private static final long serialVersionUID = 1L;
-		
-		private int intField;
-		private T tField;
-		
-		public DummyCustomParameterizedType(int intF, T tF) {
-			this.intField = intF;
-			this.tField = tF;
-		}
-		
-		public DummyCustomParameterizedType() {
-			this.intField = 0;
-			this.tField = null;
-		}
-
-		public int getIntField() {
-			return intField;
-		}
-		
-		public void setIntField(int intF) {
-			this.intField = intF;
-		}
-		
-		public void setTField(T tF) {
-			this.tField = tF;
-		}
-		
-		public T getTField() {
-			return tField;
-		}
-		
-		@Override
-		public String toString() {
-			return "(" + tField.toString() + "," + intField + ")";
-		}
-	}
-
-	/**
-	 * Method useful for suppressing sysout printing
-	 */
-	public static void pipeSystemOutToNull() {
-		System.setOut(new PrintStream(new BlackholeOutputSteam()));
-	}
-
-	private static final class BlackholeOutputSteam extends java.io.OutputStream {
-		@Override
-		public void write(int b){}
-	}
-
-	/**
-	 * utils for getting the second graph for the test of method difference();
-	 * @param env
-	 */
-	public static final DataSet<Edge<Long,Long>> getLongLongEdgeDataDifference(
-			ExecutionEnvironment env){
-		return env.fromCollection(getLongLongEdgesForDifference());
-	}
-
-	public static final DataSet<Edge<Long,Long>> getLongLongEdgeDataDifference2(
-			ExecutionEnvironment env){
-		return env.fromCollection(getLongLongEdgesForDifference2());
-	}
-
-	public static final DataSet<Vertex<Long,Long>> getLongLongVertexDataDifference(
-			ExecutionEnvironment env)
-	{
-		return env.fromCollection(getVerticesForDifference());
-	}
-
-	public static final List<Vertex<Long,Long>> getVerticesForDifference(){
-		List<Vertex<Long,Long>> vertices = new ArrayList<Vertex<Long,Long>>();
-		vertices.add(new Vertex<Long, Long>(1L, 1L));
-		vertices.add(new Vertex<Long, Long>(3L, 3L));
-		vertices.add(new Vertex<Long, Long>(6L, 6L));
-
-		return vertices;
-
-	}
-
-	public static final List<Edge<Long, Long>> getLongLongEdgesForDifference() {
-		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-		edges.add(new Edge<Long, Long>(1L, 3L, 13L));
-		edges.add(new Edge<Long, Long>(1L, 6L, 26L));
-		edges.add(new Edge<Long, Long>(6L, 3L, 63L));
-		return edges;
-	}
-
-	public static final List<Edge<Long, Long>> getLongLongEdgesForDifference2() {
-		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-		edges.add(new Edge<Long, Long>(6L, 6L, 66L));
-		return edges;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
deleted file mode 100644
index 0feb3fb..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/VertexCentricConfigurationITCase.java
+++ /dev/null
@@ -1,689 +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;
-
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-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.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexCentricConfiguration;
-import org.apache.flink.graph.spargel.VertexCentricIteration;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.LongValue;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.graph.utils.VertexToTuple2Map;
-
-@RunWith(Parameterized.class)
-public class VertexCentricConfigurationITCase extends MultipleProgramsTestBase {
-
-	public VertexCentricConfigurationITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testRunWithConfiguration() throws Exception {
-		/*
-		 * Test Graph's runVertexCentricIteration when configuration parameters are provided
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
-				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
-
-		// create the configuration object
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.addBroadcastSetForUpdateFunction("updateBcastSet", env.fromElements(1, 2, 3));
-		parameters.addBroadcastSetForMessagingFunction("messagingBcastSet", env.fromElements(4, 5, 6));
-		parameters.registerAggregator("superstepAggregator", new LongSumAggregator());
-		parameters.setOptNumVertices(true);
-
-		Graph<Long, Long, Long> res = graph.runVertexCentricIteration(
-				new UpdateFunction(), new MessageFunction(), 10, parameters);
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-        
-		expectedResult = "1,11\n" +
-						"2,11\n" +
-						"3,11\n" +
-						"4,11\n" +
-						"5,11";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationConfiguration() throws Exception {
-
-		/*
-		 * Test name, parallelism and solutionSetUnmanaged parameters
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		VertexCentricIteration<Long, Long, Long, Long> iteration = VertexCentricIteration
-				.withEdges(TestGraphUtils.getLongLongEdgeData(env), new DummyUpdateFunction(), 
-						new DummyMessageFunction(), 10);
-		
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-		parameters.setName("gelly iteration");
-		parameters.setParallelism(2);
-		parameters.setSolutionSetUnmanagedMemory(true);
-		
-		iteration.configure(parameters);
-		
-		Assert.assertEquals("gelly iteration", iteration.getIterationConfiguration().getName(""));
-		Assert.assertEquals(2, iteration.getIterationConfiguration().getParallelism());
-		Assert.assertEquals(true, iteration.getIterationConfiguration().isSolutionSetUnmanagedMemory());
-
-		DataSet<Vertex<Long, Long>> data = TestGraphUtils.getLongLongVertexData(env).runOperation(iteration);
-        List<Vertex<Long,Long>> result= data.collect();
-        
-		expectedResult = "1,11\n" +
-						"2,12\n" +
-						"3,13\n" +
-						"4,14\n" +
-						"5,15";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testDefaultConfiguration() throws Exception {
-		/*
-		 * Test Graph's runVertexCentricIteration when configuration parameters are not provided
-		 * i.e. degrees and numVertices will be -1, EdgeDirection will be OUT.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(), 
-				TestGraphUtils.getLongLongEdges(), env).mapVertices(new AssignOneMapper());
-
-		Graph<Long, Long, Long> res = graph.runVertexCentricIteration(
-				new UpdateFunctionDefault(), new MessageFunctionDefault(), 5);
-
-		
-		DataSet<Tuple2<Long, Long>> data = res.getVertices().map(new VertexToTuple2Map<Long, Long>());
-        List<Tuple2<Long, Long>> result= data.collect();
-        
-		expectedResult = "1,6\n" +
-						"2,6\n" +
-						"3,6\n" +
-						"4,6\n" +
-						"5,6";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationDefaultDirection() throws Exception {
-
-		/*
-		 * Test that if no direction parameter is given, the iteration works as before
-		 * (i.e. it collects messages from the in-neighbors and sends them to the out-neighbors)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
-				.mapVertices(new InitialiseHashSetMapper());
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
-				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerTrg(), 5)
-				.getVertices();
-
-        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
-
-		expectedResult = "1,[5]\n" +
-				"2,[1]\n" +
-				"3,[1, 2]\n" +
-				"4,[3]\n" +
-				"5,[3, 4]";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationINDirection() throws Exception {
-
-		/*
-		 * Test that if the direction parameter is set to IN,
-		 * messages are collected from the out-neighbors and sent to the in-neighbors.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
-				.mapVertices(new InitialiseHashSetMapper());
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.setDirection(EdgeDirection.IN);
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
-				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerSrc(), 5, parameters)
-				.getVertices();
-
-        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
-
-		expectedResult = "1,[2, 3]\n" +
-				"2,[3]\n" +
-				"3,[4, 5]\n" +
-				"4,[5]\n" +
-				"5,[1]";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testIterationALLDirection() throws Exception {
-
-		/*
-		 * Test that if the direction parameter is set to ALL,
-		 * messages are collected from all the neighbors and sent to all the neighbors.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, HashSet<Long>, Long> graph = Graph
-				.fromCollection(TestGraphUtils.getLongLongVertices(), TestGraphUtils.getLongLongEdges(), env)
-				.mapVertices(new InitialiseHashSetMapper());
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.setDirection(EdgeDirection.ALL);
-
-		DataSet<Vertex<Long, HashSet<Long>>> resultedVertices = graph
-				.runVertexCentricIteration(new VertexUpdateDirection(), new IdMessengerAll(), 5, parameters)
-				.getVertices();
-
-        List<Vertex<Long, HashSet<Long>>> result= resultedVertices.collect();
-
-		expectedResult = "1,[2, 3, 5]\n" +
-				"2,[1, 3]\n" +
-				"3,[1, 2, 4, 5]\n" +
-				"4,[3, 5]\n" +
-				"5,[1, 3, 4]";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testNumVerticesNotSet() throws Exception {
-
-		/*
-		 * Test that if the number of vertices option is not set, -1 is returned as value.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		DataSet<Vertex<Long, Long>> verticesWithNumVertices = graph.runVertexCentricIteration(new UpdateFunctionNumVertices(),
-				new DummyMessageFunction(), 2).getVertices();
-
-        List<Vertex<Long, Long>> result= verticesWithNumVertices.collect();
-
-		expectedResult = "1,-1\n" +
-				"2,-1\n" +
-				"3,-1\n" +
-				"4,-1\n" +
-				"5,-1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testInDegreesSet() throws Exception {
-
-		/*
-		 * Test that if the degrees are set, they can be accessed in every superstep 
-		 * inside the update function and the value
-		 * is correctly computed for degrees in the messaging function.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.setOptDegrees(true);
-
-		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
-				new UpdateFunctionInDegrees(), new DegreesMessageFunction(), 5, parameters).getVertices();
-
-        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
-
-		expectedResult = "1,1\n" +
-				"2,1\n" +
-				"3,2\n" +
-				"4,1\n" +
-				"5,2";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testInDegreesNotSet() throws Exception {
-
-		/*
-		 * Test that if the degrees option is not set, then -1 is returned as a value for in-degree.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
-				new UpdateFunctionInDegrees(), new DummyMessageFunction(), 2).getVertices();
-
-        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
-
-		expectedResult = "1,-1\n" +
-				"2,-1\n" +
-				"3,-1\n" +
-				"4,-1\n" +
-				"5,-1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testOutDegreesSet() throws Exception {
-
-		/*
-		 * Test that if the degrees are set, they can be accessed in every superstep
-		 * inside the update function and the value
-		 * is correctly computed for degrees in the messaging function.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.setOptDegrees(true);
-
-		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
-				new UpdateFunctionOutDegrees(), new DegreesMessageFunction(), 5, parameters).getVertices();
-
-        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
-
-		expectedResult = "1,2\n" +
-				"2,1\n" +
-				"3,2\n" +
-				"4,1\n" +
-				"5,1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testOutDegreesNotSet() throws Exception {
-
-		/*
-		 * Test that if the degrees option is not set, then -1 is returned as a value for out-degree.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		DataSet<Vertex<Long, Long>> verticesWithDegrees = graph.runVertexCentricIteration(
-				new UpdateFunctionInDegrees(), new DummyMessageFunction(), 2).getVertices();
-
-        List<Vertex<Long, Long>> result= verticesWithDegrees.collect();
-
-		expectedResult = "1,-1\n" +
-				"2,-1\n" +
-				"3,-1\n" +
-				"4,-1\n" +
-				"5,-1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testDirectionALLAndDegrees() throws Exception {
-
-		/*
-		 * Compute the number of neighbors in a vertex - centric manner, and verify that it is equal to
-		 * the sum: inDegree + outDegree.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Boolean, Long> graph = Graph.fromCollection(TestGraphUtils.getLongBooleanVertices(),
-				TestGraphUtils.getLongLongEdges(), env);
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		parameters.setOptDegrees(true);
-		parameters.setDirection(EdgeDirection.ALL);
-
-		DataSet<Vertex<Long, Boolean>> verticesWithNumNeighbors = graph.runVertexCentricIteration(
-				new VertexUpdateNumNeighbors(), new IdMessenger(), 1, parameters).getVertices();
-
-        List<Vertex<Long, Boolean>> result= verticesWithNumNeighbors.collect();
-
-		expectedResult = "1,true\n" +
-				"2,true\n" +
-				"3,true\n" +
-				"4,true\n" +
-				"5,true";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
-
-		LongSumAggregator aggregator = new LongSumAggregator();
-
-		@Override
-		public void preSuperstep() {
-			
-			// test bcast variable
-			@SuppressWarnings("unchecked")
-			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("updateBcastSet");
-			Assert.assertEquals(1, bcastSet.get(0));
-			Assert.assertEquals(2, bcastSet.get(1));
-			Assert.assertEquals(3, bcastSet.get(2));
-			
-			// test aggregator
-			aggregator = getIterationAggregator("superstepAggregator");
-
-			// test number of vertices
-			Assert.assertEquals(5, getNumberOfVertices());
-			
-		}
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			long superstep = getSuperstepNumber();
-			aggregator.aggregate(superstep);
-
-			setNewVertexValue(vertex.getValue() + 1);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunctionDefault extends VertexUpdateFunction<Long, Long, Long> {
-
-		LongSumAggregator aggregator = new LongSumAggregator();
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-
-			// test number of vertices
-			Assert.assertEquals(-1, getNumberOfVertices());
-
-			// test degrees
-			Assert.assertEquals(-1, getInDegree());
-			Assert.assertEquals(-1, getOutDegree());
-
-			setNewVertexValue(vertex.getValue() + 1);
-		}
-	}
-	
-	@SuppressWarnings("serial")
-	public static final class MessageFunction extends MessagingFunction<Long, Long, Long, Long> {
-
-		@Override
-		public void preSuperstep() {
-			
-			// test bcast variable
-			@SuppressWarnings("unchecked")
-			List<Tuple1<Integer>> bcastSet = (List<Tuple1<Integer>>)(List<?>)getBroadcastSet("messagingBcastSet");
-			Assert.assertEquals(4, bcastSet.get(0));
-			Assert.assertEquals(5, bcastSet.get(1));
-			Assert.assertEquals(6, bcastSet.get(2));
-
-			// test number of vertices
-			Assert.assertEquals(5, getNumberOfVertices());
-			
-			// test aggregator
-			if (getSuperstepNumber() == 2) {
-				long aggrValue = ((LongValue)getPreviousIterationAggregate("superstepAggregator")).getValue();
-				Assert.assertEquals(5, aggrValue);
-			}
-		}
-
-		@Override
-		public void sendMessages(Vertex<Long, Long> vertex) {
-			//send message to keep vertices active
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class MessageFunctionDefault extends MessagingFunction<Long, Long, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, Long> vertex) {
-			// test number of vertices
-			Assert.assertEquals(-1, getNumberOfVertices());
-
-			// test degrees
-			Assert.assertEquals(-1, getInDegree());
-			Assert.assertEquals(-1, getOutDegree());
-			//send message to keep vertices active
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunctionNumVertices extends VertexUpdateFunction<Long, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-				setNewVertexValue(getNumberOfVertices());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class DummyUpdateFunction extends VertexUpdateFunction<Long, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			setNewVertexValue(vertex.getValue() + 1);
-		}
-	}
-	
-	@SuppressWarnings("serial")
-	public static final class DummyMessageFunction extends MessagingFunction<Long, Long, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, Long> vertex) {
-			//send message to keep vertices active
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class DegreesMessageFunction extends MessagingFunction<Long, Long, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, Long> vertex) {
-			if (vertex.getId().equals(1)) {
-				Assert.assertEquals(2, getOutDegree());
-				Assert.assertEquals(1, getInDegree());
-			}
-			else if(vertex.getId().equals(3)) {
-				Assert.assertEquals(2, getOutDegree());
-				Assert.assertEquals(2, getInDegree());
-			}
-			//send message to keep vertices active
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class VertexUpdateDirection extends VertexUpdateFunction<Long, HashSet<Long>, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, HashSet<Long>> vertex, MessageIterator<Long> messages) throws Exception {
-			vertex.getValue().clear();
-
-			for(long msg : messages) {
-				vertex.getValue().add(msg);
-			}
-
-			setNewVertexValue(vertex.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunctionInDegrees extends VertexUpdateFunction<Long, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			long inDegree = getInDegree();
-			setNewVertexValue(inDegree);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunctionOutDegrees extends VertexUpdateFunction<Long, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			long outDegree = getOutDegree();
-			setNewVertexValue(outDegree);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class VertexUpdateNumNeighbors extends VertexUpdateFunction<Long, Boolean,
-			Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Boolean> vertex, MessageIterator<Long> messages) throws Exception {
-
-			long count = 0;
-
-			for(@SuppressWarnings("unused") long msg : messages) {
-				count++;
-			}
-			setNewVertexValue(count == (getInDegree() + getOutDegree()));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class UpdateFunctionDegrees extends VertexUpdateFunction<Long, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<Long, Long> vertex, MessageIterator<Long> inMessages) {
-			long inDegree = getInDegree();
-			long outDegree = getOutDegree();
-			setNewVertexValue(inDegree + outDegree);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class IdMessengerSrc extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
-			for (Edge<Long, Long> edge : getEdges()) {
-				sendMessageTo(edge.getSource(), vertex.getId());
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class IdMessengerAll extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
-			for (Edge<Long, Long> edge : getEdges()) {
-				if(edge.getSource() != vertex.getId()) {
-					sendMessageTo(edge.getSource(), vertex.getId());
-				} else {
-					sendMessageTo(edge.getTarget(), vertex.getId());
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class IdMessenger extends MessagingFunction<Long, Boolean, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, Boolean> vertex) throws Exception {
-			for (Edge<Long, Long> edge : getEdges()) {
-				if(edge.getSource() != vertex.getId()) {
-					sendMessageTo(edge.getSource(), vertex.getId());
-				} else {
-					sendMessageTo(edge.getTarget(), vertex.getId());
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class IdMessengerTrg extends MessagingFunction<Long, HashSet<Long>, Long, Long> {
-
-		@Override
-		public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
-			for (Edge<Long, Long> edge : getEdges()) {
-				sendMessageTo(edge.getTarget(), vertex.getId());
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class AssignOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
-
-		public Long map(Vertex<Long, Long> value) {
-			return 1l;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class InitialiseHashSetMapper implements MapFunction<Vertex<Long, Long>, HashSet<Long>> {
-
-		@Override
-		public HashSet<Long> map(Vertex<Long, Long> value) throws Exception {
-			return new HashSet<Long>();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java
deleted file mode 100644
index b0bacc4..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/ConnectedComponentsITCase.java
+++ /dev/null
@@ -1,71 +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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.graph.example.ConnectedComponents;
-import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-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 {
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
deleted file mode 100644
index 183c429..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/EuclideanGraphWeighingITCase.java
+++ /dev/null
@@ -1,77 +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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.graph.example.EuclideanGraphWeighing;
-import org.apache.flink.graph.example.utils.EuclideanGraphData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-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 EuclideanGraphWeighingITCase extends MultipleProgramsTestBase {
-
-	private String verticesPath;
-
-	private String edgesPath;
-
-	private String resultPath;
-
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public EuclideanGraphWeighingITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		resultPath = tempFolder.newFile().toURI().toString();
-		File verticesFile = tempFolder.newFile();
-		Files.write(EuclideanGraphData.VERTICES, verticesFile, Charsets.UTF_8);
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(EuclideanGraphData.EDGES, edgesFile, Charsets.UTF_8);
-
-		verticesPath = verticesFile.toURI().toString();
-		edgesPath = edgesFile.toURI().toString();
-	}
-
-	@Test
-	public void testGraphWeightingWeighing() throws Exception {
-		EuclideanGraphWeighing.main(new String[]{verticesPath, edgesPath, resultPath});
-		expected = EuclideanGraphData.RESULTED_WEIGHTED_EDGES;
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
deleted file mode 100644
index c19411b..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/IncrementalSSSPITCase.java
+++ /dev/null
@@ -1,133 +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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.example.IncrementalSSSP;
-import org.apache.flink.graph.example.utils.IncrementalSSSPData;
-import org.apache.flink.graph.spargel.VertexCentricConfiguration;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-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 IncrementalSSSPITCase extends MultipleProgramsTestBase {
-
-	private String verticesPath;
-
-	private String edgesPath;
-
-	private String edgesInSSSPPath;
-
-	private String resultPath;
-
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public IncrementalSSSPITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		resultPath = tempFolder.newFile().toURI().toString();
-		File verticesFile = tempFolder.newFile();
-		Files.write(IncrementalSSSPData.VERTICES, verticesFile, Charsets.UTF_8);
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(IncrementalSSSPData.EDGES, edgesFile, Charsets.UTF_8);
-
-		File edgesInSSSPFile = tempFolder.newFile();
-		Files.write(IncrementalSSSPData.EDGES_IN_SSSP, edgesInSSSPFile, Charsets.UTF_8);
-
-		verticesPath = verticesFile.toURI().toString();
-		edgesPath = edgesFile.toURI().toString();
-		edgesInSSSPPath = edgesInSSSPFile.toURI().toString();
-	}
-
-	@Test
-	public void testIncrementalSSSP() throws Exception {
-		IncrementalSSSP.main(new String[]{verticesPath, edgesPath, edgesInSSSPPath,
-				IncrementalSSSPData.SRC_EDGE_TO_BE_REMOVED, IncrementalSSSPData.TRG_EDGE_TO_BE_REMOVED,
-				IncrementalSSSPData.VAL_EDGE_TO_BE_REMOVED,resultPath, IncrementalSSSPData.NUM_VERTICES + ""});
-		expected = IncrementalSSSPData.RESULTED_VERTICES;
-	}
-
-	@Test
-	public void testIncrementalSSSPNonSPEdge() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Vertex<Long, Double>> vertices = IncrementalSSSPData.getDefaultVertexDataSet(env);
-		DataSet<Edge<Long, Double>> edges = IncrementalSSSPData.getDefaultEdgeDataSet(env);
-		DataSet<Edge<Long, Double>> edgesInSSSP = IncrementalSSSPData.getDefaultEdgesInSSSP(env);
-		// the edge to be removed is a non-SP edge
-		Edge<Long, Double> edgeToBeRemoved = new Edge<Long, Double>(3L, 5L, 5.0);
-
-		Graph<Long, Double, Double> graph = Graph.fromDataSet(vertices, edges, env);
-		// Assumption: all minimum weight paths are kept
-		Graph<Long, Double, Double> ssspGraph = Graph.fromDataSet(vertices, edgesInSSSP, env);
-		// remove the edge
-		graph.removeEdge(edgeToBeRemoved);
-
-		// configure the iteration
-		VertexCentricConfiguration parameters = new VertexCentricConfiguration();
-
-		if(IncrementalSSSP.isInSSSP(edgeToBeRemoved, edgesInSSSP)) {
-
-			parameters.setDirection(EdgeDirection.IN);
-			parameters.setOptDegrees(true);
-
-			// run the vertex centric iteration to propagate info
-			Graph<Long, Double, Double> result = ssspGraph.runVertexCentricIteration(
-					new IncrementalSSSP.VertexDistanceUpdater(),
-					new IncrementalSSSP.InvalidateMessenger(edgeToBeRemoved),
-					IncrementalSSSPData.NUM_VERTICES, parameters);
-
-			DataSet<Vertex<Long, Double>> resultedVertices = result.getVertices();
-
-			resultedVertices.writeAsCsv(resultPath, "\n", ",");
-			env.execute();
-		} else {
-			vertices.writeAsCsv(resultPath, "\n", ",");
-			env.execute();
-		}
-
-		expected = IncrementalSSSPData.VERTICES;
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.java
deleted file mode 100644
index 294a756..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/example/JaccardSimilarityMeasureITCase.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.example;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.graph.example.JaccardSimilarityMeasure;
-import org.apache.flink.graph.example.utils.JaccardSimilarityMeasureData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-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 JaccardSimilarityMeasureITCase extends MultipleProgramsTestBase {
-
-	private String edgesPath;
-
-	private String resultPath;
-
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	public JaccardSimilarityMeasureITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void before() throws Exception {
-		resultPath = tempFolder.newFile().toURI().toString();
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(JaccardSimilarityMeasureData.EDGES, edgesFile, Charsets.UTF_8);
-
-		edgesPath = edgesFile.toURI().toString();
-	}
-
-	@Test
-	public void testJaccardSimilarityMeasureExample() throws Exception {
-		JaccardSimilarityMeasure.main(new String[]{edgesPath, resultPath});
-		expected = JaccardSimilarityMeasureData.JACCARD_EDGES;
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected, resultPath);
-	}
-}


[22/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
new file mode 100644
index 0000000..2e51d90
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/MapVerticesITCase.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.Vertex
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class MapVerticesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
+MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithSameValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.mapVertices(new AddOneMapper).getVertices.collect().toList
+    expectedResult = "1,2\n" +
+      "2,3\n" +
+      "3,4\n" +
+      "4,5\n" +
+      "5,6\n";
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testWithSameValueSugar {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.mapVertices(vertex => vertex.getValue + 1).getVertices.collect().toList
+    expectedResult = "1,2\n" +
+      "2,3\n" +
+      "3,4\n" +
+      "4,5\n" +
+      "5,6\n";
+    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
+  }
+
+  final class AddOneMapper extends MapFunction[Vertex[Long, Long], Long] {
+    @throws(classOf[Exception])
+    def map(vertex: Vertex[Long, Long]): Long = {
+      vertex.getValue + 1
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
new file mode 100644
index 0000000..dcd1deb
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnEdgesMethodsITCase.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.scala.{EdgesFunction, EdgesFunctionWithVertexValue, Graph}
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.apache.flink.util.Collector
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class ReduceOnEdgesMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode)
+  extends MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testAllNeighborsWithValueGreaterThanFour {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour,
+      EdgeDirection.ALL).collect().toList
+    expectedResult = "(5,1)\n" + "(5,3)\n" + "(5,4)"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+
+  @Test
+  @throws(classOf[Exception])
+  def testAllNeighbors {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.groupReduceOnEdges(new SelectNeighbors, EdgeDirection.ALL)
+    .collect().toList
+    expectedResult = "(1,2)\n" + "(1,3)\n" + "(1,5)\n" + "(2,1)\n" + "(2,3)\n" +
+      "(3,1)\n" + "(3,2)\n" + "(3,4)\n" + "(3,5)\n" + "(4,3)\n" + "(4,5)\n" +
+      "(5,1)\n" + "(5,3)\n" + "(5,4)"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testLowestWeightOutNeighborNoValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new
+        SelectMinWeightNeighborNoValue, EdgeDirection.OUT)
+    val res = verticesWithLowestOutNeighbor.collect().toList
+    expectedResult = "(1,12)\n" + "(2,23)\n" + "(3,34)\n" + "(4,45)\n" + "(5,51)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testLowestWeightInNeighborNoValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val verticesWithLowestOutNeighbor: DataSet[(Long, Long)] = graph.reduceOnEdges(new
+        SelectMinWeightNeighborNoValue, EdgeDirection.IN)
+    val res = verticesWithLowestOutNeighbor.collect().toList
+    expectedResult = "(1,51)\n" + "(2,12)\n" + "(3,13)\n" + "(4,34)\n" + "(5,35)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testMaxWeightAllNeighbors {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val verticesWithMaxEdgeWeight: DataSet[(Long, Long)] = graph.reduceOnEdges(new
+        SelectMaxWeightNeighborNoValue, EdgeDirection.ALL)
+    val res = verticesWithMaxEdgeWeight.collect().toList
+    expectedResult = "(1,51)\n" + "(2,23)\n" + "(3,35)\n" + "(4,45)\n" + "(5,51)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  final class SelectNeighborsValueGreaterThanFour extends EdgesFunctionWithVertexValue[Long,
+    Long, Long, (Long, Long)] {
+    @throws(classOf[Exception])
+    override def iterateEdges(v: Vertex[Long, Long], edges: Iterable[Edge[Long, Long]], out:
+    Collector[(Long, Long)]): Unit = {
+      for (edge <- edges) {
+        if (v.getValue > 4) {
+          if (v.getId == edge.getTarget) {
+            out.collect((v.getId, edge.getSource))
+          }
+          else {
+            out.collect((v.getId, edge.getTarget))
+          }
+        }
+      }
+    }
+  }
+
+  final class SelectNeighbors extends EdgesFunction[Long, Long, (Long, Long)] {
+    @throws(classOf[Exception])
+    override def iterateEdges(edges: Iterable[(Long, Edge[Long, Long])], out: Collector[
+      (Long, Long)]) {
+      for (edge <- edges) {
+        if (edge._1.equals(edge._2.getTarget)) {
+          out.collect(new Tuple2[Long, Long](edge._1, edge._2.getSource))
+        }
+        else {
+          out.collect(new Tuple2[Long, Long](edge._1, edge._2.getTarget))
+        }
+      }
+    }
+  }
+
+  final class SelectMinWeightNeighborNoValue extends ReduceEdgesFunction[Long] {
+    override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = {
+      Math.min(firstEdgeValue, secondEdgeValue)
+    }
+  }
+
+  final class SelectMaxWeightNeighborNoValue extends ReduceEdgesFunction[Long] {
+    override def reduceEdges(firstEdgeValue: Long, secondEdgeValue: Long): Long = {
+      Math.max(firstEdgeValue, secondEdgeValue)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
new file mode 100644
index 0000000..aef5493
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/ReduceOnNeighborMethodsITCase.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.scala.test.operations
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala.test.TestGraphUtils
+import org.apache.flink.graph.scala.{NeighborsFunctionWithVertexValue, _}
+import org.apache.flink.graph.{Edge, EdgeDirection, ReduceNeighborsFunction, Vertex}
+import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
+import org.apache.flink.util.Collector
+import org.junit.rules.TemporaryFolder
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{After, Before, Rule, Test}
+import _root_.scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class ReduceOnNeighborMethodsITCase(mode: MultipleProgramsTestBase.TestExecutionMode)
+  extends MultipleProgramsTestBase(mode) {
+
+  private var expectedResult: String = null
+
+  @Test
+  @throws(classOf[Exception])
+  def testSumOfAllNeighborsNoValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.ALL)
+    .collect().toList
+    expectedResult = "(1,10)\n" + "(2,4)\n" + "(3,12)\n" + "(4,8)\n" + "(5,8)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testSumOfOutNeighborsNoValue {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val res = graph.reduceOnNeighbors(new SumNeighbors, EdgeDirection.OUT).collect().toList
+    expectedResult = "(1,5)\n" + "(2,3)\n" + "(3,9)\n" + "(4,5)\n" + "(5,1)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testSumOfAllNeighbors {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result = graph.groupReduceOnNeighbors(new SumAllNeighbors, EdgeDirection.ALL)
+    val res = result.collect().toList
+    expectedResult = "(1,11)\n" + "(2,6)\n" + "(3,15)\n" + "(4,12)\n" + "(5,13)\n"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  @Test
+  @throws(classOf[Exception])
+  def testSumOfInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
+      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
+    val result = graph.groupReduceOnNeighbors(new
+        SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo, EdgeDirection.IN)
+    val res = result.collect().toList
+    expectedResult = "(3,59)\n" + "(3,118)\n" + "(4,204)\n" + "(4,102)\n" + "(5,570)\n" + "(5,285)"
+    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
+  }
+
+  final class SumNeighbors extends ReduceNeighborsFunction[Long] {
+    override def reduceNeighbors(firstNeighbor: Long, secondNeighbor: Long): Long = {
+      firstNeighbor + secondNeighbor
+    }
+  }
+
+  final class SumAllNeighbors extends NeighborsFunctionWithVertexValue[Long, Long, Long, (Long,
+    Long)] {
+    @throws(classOf[Exception])
+    def iterateNeighbors(vertex: Vertex[Long, Long], neighbors: Iterable[(Edge[Long, Long],
+      Vertex[Long, Long])], out: Collector[(Long, Long)]) {
+      var sum: Long = 0
+      for (neighbor <- neighbors) {
+        sum += neighbor._2.getValue
+      }
+      out.collect((vertex.getId, sum + vertex.getValue))
+    }
+  }
+
+  final class SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo extends
+  NeighborsFunction[Long, Long, Long, (Long, Long)] {
+    @throws(classOf[Exception])
+    def iterateNeighbors(neighbors: Iterable[(Long, Edge[Long, Long], Vertex[Long, Long])],
+                         out: Collector[(Long, Long)]) {
+      var sum: Long = 0
+      var next: (Long, Edge[Long, Long], Vertex[Long, Long]) = null
+      val neighborsIterator: Iterator[(Long, Edge[Long, Long], Vertex[Long, Long])] =
+        neighbors.iterator
+      while (neighborsIterator.hasNext) {
+        next = neighborsIterator.next
+        sum += next._3.getValue * next._2.getValue
+      }
+      if (next._1 > 2) {
+        out.collect(new Tuple2[Long, Long](next._1, sum))
+        out.collect(new Tuple2[Long, Long](next._1, sum * 2))
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml
new file mode 100644
index 0000000..7fd95ed
--- /dev/null
+++ b/flink-libraries/flink-gelly/pom.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-libraries</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	
+	<artifactId>flink-gelly</artifactId>
+	<name>flink-gelly</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-optimizer</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
new file mode 100644
index 0000000..d84badb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Edge.java
@@ -0,0 +1,75 @@
+/*
+ * 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;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+
+/**
+ * An Edge represents a link between two {@link Vertex vertices},
+ * the source and the target and can carry an attached value.
+ * For edges with no value, use {@link org.apache.flink.types.NullValue} as the value type.
+ *
+ * @param <K> the key type for the sources and target vertices
+ * @param <V> the edge value type
+ */
+public class Edge<K, V> extends Tuple3<K, K, V>{
+
+	private static final long serialVersionUID = 1L;
+
+	public Edge(){}
+
+	public Edge(K src, K trg, V val) {
+		this.f0 = src;
+		this.f1 = trg;
+		this.f2 = val;
+	}
+
+	/**
+	 * Reverses the direction of this Edge.
+	 * @return a new Edge, where the source is the original Edge's target
+	 * and the target is the original Edge's source.
+	 */
+	public Edge<K, V> reverse() {
+			return new Edge<K, V>(this.f1, this.f0, this.f2);
+	}
+
+	public void setSource(K src) {
+		this.f0 = src;
+	}
+
+	public K getSource() {
+		return this.f0;
+	}
+
+	public void setTarget(K target) {
+		this.f1 = target;
+	}
+
+	public K getTarget() {
+		return f1;
+	}
+
+	public void setValue(V value) {
+		this.f2 = value;
+	}
+
+	public V getValue() {
+		return f2;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
new file mode 100644
index 0000000..0a055bb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgeDirection.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * The EdgeDirection is used to select a node's neighborhood
+ * by the {@link Graph#groupReduceOnEdges(EdgesFunction, EdgeDirection)},
+ * {@link Graph#groupReduceOnEdges(EdgesFunctionWithVertexValue, EdgeDirection)},
+ * {@link Graph#groupReduceOnNeighbors(NeighborsFunction, EdgeDirection)},
+ * {@link Graph#groupReduceOnNeighbors(NeighborsFunctionWithVertexValue, EdgeDirection)},
+ * {@link Graph#reduceOnEdges(ReduceEdgesFunction, EdgeDirection)} and
+ * {@link Graph#reduceOnNeighbors(ReduceNeighborsFunction, EdgeDirection)}
+ * methods.
+ */
+public enum EdgeDirection {
+	IN,
+	OUT,
+	ALL
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
new file mode 100644
index 0000000..bf1d6a2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.Collector;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#groupReduceOnEdges(EdgesFunction, EdgeDirection)} method.
+ *
+ * @param <K> the vertex key type
+ * @param <EV> the edge value type
+ * @param <O> the type of the return value
+ */
+public interface EdgesFunction<K, EV, O> extends Function, Serializable {
+
+	void iterateEdges(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<O> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
new file mode 100644
index 0000000..0b0ab0e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/EdgesFunctionWithVertexValue.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.util.Collector;
+
+/**
+ * Interface to be implemented by the function applied to a vertex neighborhood
+ * in the {@link Graph#groupReduceOnEdges(EdgesFunctionWithVertexValue, EdgeDirection)}
+ * method.
+ *
+ * @param <K> the vertex key type
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ * @param <O> the type of the return value
+ */
+public interface EdgesFunctionWithVertexValue<K, VV, EV, O> extends Function, Serializable {
+
+	void iterateEdges(Vertex<K, VV> v, Iterable<Edge<K, EV>> edges, Collector<O> out) throws Exception;
+}


[13/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
new file mode 100644
index 0000000..7553b32
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborMethodsITCase.java
@@ -0,0 +1,668 @@
+/*
+ * 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.operations;
+
+import java.util.Iterator;
+import java.util.List;
+
+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.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.NeighborsFunction;
+import org.apache.flink.graph.NeighborsFunctionWithVertexValue;
+import org.apache.flink.graph.ReduceNeighborsFunction;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ReduceOnNeighborMethodsITCase extends MultipleProgramsTestBase {
+
+	public ReduceOnNeighborMethodsITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testSumOfOutNeighbors() throws Exception {
+		/*
+		 * Get the sum of out-neighbor values
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
+				graph.groupReduceOnNeighbors(new SumOutNeighbors(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+		
+		expectedResult = "1,5\n" +
+				"2,3\n" + 
+				"3,9\n" +
+				"4,5\n" + 
+				"5,1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfInNeighbors() throws Exception {
+		/*
+		 * Get the sum of in-neighbor values
+		 * times the edge weights for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSum = 
+				graph.groupReduceOnNeighbors(new SumInNeighbors(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
+		
+		expectedResult = "1,255\n" +
+				"2,12\n" + 
+				"3,59\n" +
+				"4,102\n" + 
+				"5,285\n";
+		
+		compareResultAsTuples(result, expectedResult);
+		
+		
+	}
+
+	@Test
+	public void testSumOfOAllNeighbors() throws Exception {
+		/*
+		 * Get the sum of all neighbor values
+		 * including own vertex value
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
+				graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "1,11\n" +
+				"2,6\n" + 
+				"3,15\n" +
+				"4,12\n" + 
+				"5,13\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOutNeighborsIdGreaterThanThree() throws Exception {
+		/*
+		 * Get the sum of out-neighbor values
+		 * for each vertex with id greater than three.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumOutNeighborsIdGreaterThanThree(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+		
+		expectedResult = "4,5\n" +
+				"5,1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfInNeighborsIdGreaterThanThree() throws Exception {
+		/*
+		 * Get the sum of in-neighbor values
+		 * times the edge weights for each vertex with id greater than three.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSum =
+				graph.groupReduceOnNeighbors(new SumInNeighborsIdGreaterThanThree(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
+		
+		expectedResult = "4,102\n" +
+				"5,285\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOAllNeighborsIdGreaterThanThree() throws Exception {
+		/*
+		 * Get the sum of all neighbor values
+		 * including own vertex value
+		 * for each vertex with id greater than three.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumAllNeighborsIdGreaterThanThree(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "4,12\n" +
+				"5,13\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOutNeighborsNoValue() throws Exception {
+		/*
+		 * Get the sum of out-neighbor values
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues = 
+				graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "1,5\n" +
+				"2,3\n" + 
+				"3,9\n" +
+				"4,5\n" + 
+				"5,1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfInNeighborsNoValue() throws Exception {
+		/*
+		 * Get the sum of in-neighbor values
+		 * times the edge weights for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSum = 
+				graph.groupReduceOnNeighbors(new SumInNeighborsNoValue(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
+	
+		expectedResult = "1,255\n" +
+				"2,12\n" +
+				"3,59\n" +
+				"4,102\n" +
+				"5,285\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfAllNeighborsNoValue() throws Exception {
+		/*
+		 * Get the sum of all neighbor values
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
+				graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfAllNeighborValues.collect();
+	
+		expectedResult = "1,10\n" +
+				"2,4\n" + 
+				"3,12\n" +
+				"4,8\n" + 
+				"5,8\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
+		/*
+		 * Get the sum of out-neighbor values
+		 * for each vertex with id greater than two as well as the same sum multiplied by two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "3,9\n" +
+				"3,18\n" +
+				"4,5\n" +
+				"4,10\n" +
+				"5,1\n" +
+				"5,2";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
+		/*
+		 * Get the sum of in-neighbor values
+		 * for each vertex with id greater than two as well as the same sum multiplied by two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "3,59\n" +
+				"3,118\n" +
+				"4,204\n" +
+				"4,102\n" +
+				"5,570\n" +
+				"5,285";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo() throws Exception {
+		/*
+		 * Get the sum of all neighbor values
+		 * for each vertex with id greater than two as well as the same sum multiplied by two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
+				graph.groupReduceOnNeighbors(new SumAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfAllNeighborValues.collect();
+
+		expectedResult = "3,12\n" +
+				"3,24\n" +
+				"4,8\n" +
+				"4,16\n" +
+				"5,8\n" +
+				"5,16";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOutNeighborsMultipliedByTwo() throws Exception {
+		/*
+		 * Get the sum of out-neighbor values
+		 * for each vertex as well as the sum of out-neighbor values multiplied by two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumOutNeighborsMultipliedByTwo(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+		
+		expectedResult = "1,5\n" +
+				"1,10\n" +
+				"2,3\n" +
+				"2,6\n" +
+				"3,9\n" +
+				"3,18\n" +
+				"4,5\n" +
+				"4,10\n" +
+				"5,1\n" +
+				"5,2";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfInNeighborsSubtractOne() throws Exception {
+		/*
+		 * Get the sum of in-neighbor values
+		 * times the edge weights for each vertex as well as the same sum minus one.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSum =
+				graph.groupReduceOnNeighbors(new SumInNeighborsSubtractOne(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithSum.collect();
+		
+		expectedResult = "1,255\n" +
+				"1,254\n" +
+				"2,12\n" +
+				"2,11\n" +
+				"3,59\n" +
+				"3,58\n" +
+				"4,102\n" +
+				"4,101\n" +
+				"5,285\n" +
+				"5,284";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testSumOfOAllNeighborsAddFive() throws Exception {
+		/*
+		 * Get the sum of all neighbor values
+		 * including own vertex value
+		 * for each vertex as well as the same sum plus five.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+				graph.groupReduceOnNeighbors(new SumAllNeighborsAddFive(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithSumOfOutNeighborValues.collect();
+
+		expectedResult = "1,11\n" +
+				"1,16\n" +
+				"2,6\n" +
+				"2,11\n" +
+				"3,15\n" +
+				"3,20\n" +
+				"4,12\n" +
+				"4,17\n" +
+				"5,13\n" +
+				"5,18";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumOutNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
+	Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+				Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+				Collector<Tuple2<Long, Long>> out) throws Exception {
+			
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumInNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
+		Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+				Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+				Collector<Tuple2<Long, Long>> out) throws Exception {
+		
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumAllNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long, 
+		Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+	
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumOutNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+					sum += neighbor.f1.getValue();
+			}
+			if(vertex.getId() > 3) {
+				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumInNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
+			}
+			if(vertex.getId() > 3) {
+				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumAllNeighborsIdGreaterThanThree implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			if(vertex.getId() > 3) {
+				out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumNeighbors implements ReduceNeighborsFunction<Long> {
+
+		@Override
+		public Long reduceNeighbors(Long firstNeighbor, Long secondNeighbor) {
+			return firstNeighbor + secondNeighbor;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumInNeighborsNoValue implements NeighborsFunction<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+			long sum = 0;
+			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
+			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
+					neighbors.iterator();
+			while(neighborsIterator.hasNext()) {
+				next = neighborsIterator.next();
+				sum += next.f2.getValue() * next.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(next.f0, sum));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumOutNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
+			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
+					neighbors.iterator();
+			while(neighborsIterator.hasNext()) {
+				next = neighborsIterator.next();
+				sum += next.f2.getValue();
+			}
+			if(next.f0 > 2) {
+				out.collect(new Tuple2<Long, Long>(next.f0, sum));
+				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumInNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
+			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
+					neighbors.iterator();
+			while(neighborsIterator.hasNext()) {
+				next = neighborsIterator.next();
+				sum += next.f2.getValue() * next.f1.getValue();
+			}
+			if(next.f0 > 2) {
+				out.collect(new Tuple2<Long, Long>(next.f0, sum));
+				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumAllNeighborsNoValueMultipliedByTwoIdGreaterThanTwo implements NeighborsFunction<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Iterable<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>> next = null;
+			Iterator<Tuple3<Long, Edge<Long, Long>, Vertex<Long, Long>>> neighborsIterator =
+					neighbors.iterator();
+			while(neighborsIterator.hasNext()) {
+				next = neighborsIterator.next();
+				sum += next.f2.getValue();
+			}
+			if(next.f0 > 2) {
+				out.collect(new Tuple2<Long, Long>(next.f0, sum));
+				out.collect(new Tuple2<Long, Long>(next.f0, sum * 2));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumOutNeighborsMultipliedByTwo implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum * 2));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumInNeighborsSubtractOne implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f0.getValue() * neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum));
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum - 1));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumAllNeighborsAddFive implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue() + 5));
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
new file mode 100644
index 0000000..b32abeb
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -0,0 +1,203 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.NeighborsFunctionWithVertexValue;
+import org.apache.flink.graph.ReduceNeighborsFunction;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.util.Collector;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+public class ReduceOnNeighborsWithExceptionITCase {
+
+	private static final int PARALLELISM = 4;
+
+	private static ForkableFlinkMiniCluster cluster;
+
+
+	@BeforeClass
+	public static void setupCluster() {
+		try {
+			Configuration config = new Configuration();
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail("Error starting test cluster: " + e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDownCluster() {
+		try {
+			cluster.stop();
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Cluster shutdown caused an exception: " + t.getMessage());
+		}
+	}
+
+	/**
+	 * Test groupReduceOnNeighbors() -NeighborsFunctionWithVertexValue-
+	 * with an edge having a srcId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnNeighborsWithVVInvalidEdgeSrcId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
+
+			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test groupReduceOnNeighbors() -NeighborsFunctionWithVertexValue-
+	 * with an edge having a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnNeighborsWithVVInvalidEdgeTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithSumOfOutNeighborValues =
+					graph.groupReduceOnNeighbors(new SumAllNeighbors(), EdgeDirection.ALL);
+
+			verticesWithSumOfOutNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test groupReduceOnNeighbors() -NeighborsFunction-
+	 * with an edge having a srcId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnNeighborsInvalidEdgeSrcId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
+					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
+
+			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test groupReduceOnNeighbors() -NeighborsFunction-
+	 * with an edge having a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnNeighborsInvalidEdgeTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithSumOfAllNeighborValues =
+					graph.reduceOnNeighbors(new SumNeighbors(), EdgeDirection.ALL);
+
+			verticesWithSumOfAllNeighborValues.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumAllNeighbors implements NeighborsFunctionWithVertexValue<Long, Long, Long,
+			Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateNeighbors(Vertex<Long, Long> vertex,
+									 Iterable<Tuple2<Edge<Long, Long>, Vertex<Long, Long>>> neighbors,
+									 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			long sum = 0;
+			for (Tuple2<Edge<Long, Long>, Vertex<Long, Long>> neighbor : neighbors) {
+				sum += neighbor.f1.getValue();
+			}
+			out.collect(new Tuple2<Long, Long>(vertex.getId(), sum + vertex.getValue()));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SumNeighbors implements ReduceNeighborsFunction<Long> {
+
+		@Override
+		public Long reduceNeighbors(Long firstNeighbor, Long secondNeighbor) {
+			return firstNeighbor + secondNeighbor;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml
new file mode 100644
index 0000000..b38ffeb
--- /dev/null
+++ b/flink-libraries/pom.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-libraries</artifactId>
+	<name>flink-libraries</name>
+	<packaging>pom</packaging>
+
+	<modules>
+		<module>flink-gelly</module>
+		<module>flink-gelly-scala</module>
+	</modules>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/pom.xml b/flink-staging/flink-gelly-scala/pom.xml
deleted file mode 100644
index edcb865..0000000
--- a/flink-staging/flink-gelly-scala/pom.xml
+++ /dev/null
@@ -1,204 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <groupId>org.apache.flink</groupId>
-        <artifactId>flink-staging</artifactId>
-        <version>0.10-SNAPSHOT</version>
-        <relativePath>..</relativePath>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>flink-gelly-scala</artifactId>
-
-    <packaging>jar</packaging>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-scala</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-clients</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-gelly</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-            <version>${guava.version}</version>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <!-- Scala Compiler -->
-            <plugin>
-                <groupId>net.alchim31.maven</groupId>
-                <artifactId>scala-maven-plugin</artifactId>
-                <version>3.1.4</version>
-                <executions>
-                    <!-- Run scala compiler in the process-resources phase, so that dependencies on
-                        scala classes can be resolved later in the (Java) compile phase -->
-                    <execution>
-                        <id>scala-compile-first</id>
-                        <phase>process-resources</phase>
-                        <goals>
-                            <goal>compile</goal>
-                        </goals>
-                    </execution>
-
-                    <!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-                         scala classes can be resolved later in the (Java) test-compile phase -->
-                    <execution>
-                        <id>scala-test-compile</id>
-                        <phase>process-test-resources</phase>
-                        <goals>
-                            <goal>testCompile</goal>
-                        </goals>
-                    </execution>
-                </executions>
-                <configuration>
-                    <jvmArgs>
-                        <jvmArg>-Xms128m</jvmArg>
-                        <jvmArg>-Xmx512m</jvmArg>
-                    </jvmArgs>
-                    <compilerPlugins combine.children="append">
-                        <compilerPlugin>
-                            <groupId>org.scalamacros</groupId>
-                            <artifactId>paradise_${scala.version}</artifactId>
-                            <version>${scala.macros.version}</version>
-                        </compilerPlugin>
-                    </compilerPlugins>
-                </configuration>
-            </plugin>
-
-            <!-- Eclipse Integration -->
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-eclipse-plugin</artifactId>
-                <version>2.8</version>
-                <configuration>
-                    <downloadSources>true</downloadSources>
-                    <projectnatures>
-                        <projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-                        <projectnature>org.eclipse.jdt.core.javanature</projectnature>
-                    </projectnatures>
-                    <buildcommands>
-                        <buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-                    </buildcommands>
-                    <classpathContainers>
-                        <classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-                        <classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-                    </classpathContainers>
-                    <excludes>
-                        <exclude>org.scala-lang:scala-library</exclude>
-                        <exclude>org.scala-lang:scala-compiler</exclude>
-                    </excludes>
-                    <sourceIncludes>
-                        <sourceInclude>**/*.scala</sourceInclude>
-                        <sourceInclude>**/*.java</sourceInclude>
-                    </sourceIncludes>
-                </configuration>
-            </plugin>
-
-            <!-- Adding scala source directories to build path -->
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>1.7</version>
-                <executions>
-                    <!-- Add src/main/scala to eclipse build path -->
-                    <execution>
-                        <id>add-source</id>
-                        <phase>generate-sources</phase>
-                        <goals>
-                            <goal>add-source</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>src/main/scala</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                    <!-- Add src/test/scala to eclipse build path -->
-                    <execution>
-                        <id>add-test-source</id>
-                        <phase>generate-test-sources</phase>
-                        <goals>
-                            <goal>add-test-source</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>src/test/scala</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <groupId>org.scalastyle</groupId>
-                <artifactId>scalastyle-maven-plugin</artifactId>
-                <version>0.5.0</version>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>check</goal>
-                        </goals>
-                    </execution>
-                </executions>
-                <configuration>
-                    <verbose>false</verbose>
-                    <failOnViolation>true</failOnViolation>
-                    <includeTestSourceDirectory>true</includeTestSourceDirectory>
-                    <failOnWarning>false</failOnWarning>
-                    <sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
-                    <testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
-                    <configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
-                    <outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
-                    <outputEncoding>UTF-8</outputEncoding>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
deleted file mode 100644
index 70a5fdf..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
+++ /dev/null
@@ -1,35 +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.scala
-
-import org.apache.flink.api.java.tuple.Tuple2
-import org.apache.flink.graph.Edge
-import org.apache.flink.util.Collector
-
-abstract class EdgesFunction[K, EV, T] extends org.apache.flink.graph.EdgesFunction[K, EV, T] {
-
-  def iterateEdges(edges: Iterable[(K, Edge[K, EV])], out: Collector[T])
-
-  override def iterateEdges(edges: java.lang.Iterable[Tuple2[K, Edge[K, EV]]], out:
-  Collector[T]): Unit = {
-    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(edges)
-      .map(jtuple => (jtuple.f0, jtuple.f1))
-    iterateEdges(scalaIterable, out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
deleted file mode 100644
index 82589b6..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
+++ /dev/null
@@ -1,33 +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.scala
-
-import org.apache.flink.graph.{Edge, Vertex}
-import org.apache.flink.util.Collector
-
-abstract class EdgesFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph
-.EdgesFunctionWithVertexValue[K, VV, EV, T] {
-  @throws(classOf[Exception])
-  def iterateEdges(v: Vertex[K, VV], edges: Iterable[Edge[K, EV]], out: Collector[T])
-
-  override def iterateEdges(v: Vertex[K, VV], edges: java.lang.Iterable[Edge[K, EV]], out:
-  Collector[T]) = {
-    iterateEdges(v, scala.collection.JavaConversions.iterableAsScalaIterable(edges), out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
deleted file mode 100644
index 28f3f12..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
+++ /dev/null
@@ -1,1014 +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.scala
-
-import org.apache.flink.api.common.functions.{FilterFunction, MapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.{tuple => jtuple}
-import org.apache.flink.api.scala._
-import org.apache.flink.graph._
-import org.apache.flink.graph.validation.GraphValidator
-import org.apache.flink.graph.gsa.{ApplyFunction, GSAConfiguration, GatherFunction, SumFunction}
-import org.apache.flink.graph.spargel.{MessagingFunction, VertexCentricConfiguration, VertexUpdateFunction}
-import org.apache.flink.{graph => jg}
-import _root_.scala.collection.JavaConverters._
-import _root_.scala.reflect.ClassTag
-import org.apache.flink.types.NullValue
-
-object Graph {
-
-  /**
-  * Creates a Graph from a DataSet of vertices and a DataSet of edges.
-  */
-  def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](vertices: DataSet[Vertex[K, VV]], edges: DataSet[Edge[K, EV]],
-                              env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](vertices.javaSet, edges.javaSet, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a DataSet of edges.
-  * Vertices are created automatically and their values are set to NullValue.
-  */
-  def fromDataSet[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
-  (edges: DataSet[Edge[K, EV]], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
-    wrapGraph(jg.Graph.fromDataSet[K, EV](edges.javaSet, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a graph from a DataSet of edges.
-  * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
-  */
-  def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: DataSet[Edge[K, EV]], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](edges.javaSet, mapper, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a Seq of vertices and a Seq of edges.
-  */
-  def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](vertices: Seq[Vertex[K, VV]], edges: Seq[Edge[K, EV]], env:
-  ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromCollection[K, VV, EV](vertices.asJavaCollection, edges
-      .asJavaCollection, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a Seq of edges.
-  * Vertices are created automatically and their values are set to NullValue.
-  */
-  def fromCollection[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
-  (edges: Seq[Edge[K, EV]], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
-    wrapGraph(jg.Graph.fromCollection[K, EV](edges.asJavaCollection, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a graph from a Seq of edges.
-  * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
-  */
-  def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: Seq[Edge[K, EV]], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    wrapGraph(jg.Graph.fromCollection[K, VV, EV](edges.asJavaCollection, mapper, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a DataSets of Tuples.
-  */
-  def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](vertices: DataSet[(K, VV)], edges: DataSet[(K, K, EV)],
-                              env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    val javaTupleVertices = vertices.map(v => new jtuple.Tuple2(v._1, v._2)).javaSet
-    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
-    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleVertices, javaTupleEdges,
-        env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a DataSet of Tuples representing the edges.
-  * Vertices are created automatically and their values are set to NullValue.
-  */
-  def fromTupleDataSet[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
-  (edges: DataSet[(K, K, EV)], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
-    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
-    wrapGraph(jg.Graph.fromTupleDataSet[K, EV](javaTupleEdges, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph from a DataSet of Tuples representing the edges.
-  * Vertices are created automatically and their values are set by applying the provided
-  * map function to the vertex ids.
-  */
-  def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-  TypeInformation : ClassTag](edges: DataSet[(K, K, EV)], mapper: MapFunction[K, VV],
-      env: ExecutionEnvironment): Graph[K, VV, EV] = {
-    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
-    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleEdges, mapper, env.getJavaEnv))
-  }
-
-  /**
-  * Creates a Graph with from a CSV file of vertices and a CSV file of edges
-  * 
-  * @param The Execution Environment.
-  * @param pathEdges The file path containing the edges.
-  * @param readVertices Defines whether the vertices have associated values.
-  * If set to false, the vertex input is ignored and vertices are created from the edges file.
-  * @param pathVertices The file path containing the vertices.
-  * @param hasEdgeValues Defines whether the edges have associated values. True by default.
-  * @param lineDelimiterVertices The string that separates lines in the vertices file.
-  * It defaults to newline.
-  * @param fieldDelimiterVertices The string that separates vertex Ids from vertex values
-  * in the vertices file.
-  * @param quoteCharacterVertices The character to use for quoted String parsing
-  * in the vertices file. Disabled by default.
-  * @param ignoreFirstLineVertices Whether the first line in the vertices file should be ignored.
-  * @param ignoreCommentsVertices Lines that start with the given String in the vertices file
-  * are ignored, disabled by default.
-  * @param lenientVertices Whether the parser should silently ignore malformed lines in the
-  * vertices file.
-  * @param includedFieldsVertices The fields in the vertices file that should be read.
-  * By default all fields are read.
-  * @param lineDelimiterEdges The string that separates lines in the edges file.
-  * It defaults to newline.
-  * @param fieldDelimiterEdges The string that separates fields in the edges file.
-  * @param quoteCharacterEdges The character to use for quoted String parsing
-  * in the edges file. Disabled by default.
-  * @param ignoreFirstLineEdges Whether the first line in the vertices file should be ignored.
-  * @param ignoreCommentsEdges Lines that start with the given String in the edges file
-  * are ignored, disabled by default.
-  * @param lenientEdges Whether the parser should silently ignore malformed lines in the
-  * edges file.
-  * @param includedFieldsEdges The fields in the edges file that should be read.
-  * By default all fields are read.
-  * @param mapper If no vertex values are provided, this mapper can be used to initialize them.
-  * 
-  */
-  // scalastyle:off
-  // This method exceeds the max allowed number of parameters -->  
-  def fromCsvReader[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag,
-    EV: TypeInformation : ClassTag](
-      env: ExecutionEnvironment,
-      pathEdges: String,
-      readVertices: Boolean,
-      pathVertices: String = null,
-      hasEdgeValues: Boolean = true,
-      lineDelimiterVertices: String = "\n",
-      fieldDelimiterVertices: String = ",",
-      quoteCharacterVertices: Character = null,
-      ignoreFirstLineVertices: Boolean = false,
-      ignoreCommentsVertices: String = null,
-      lenientVertices: Boolean = false,
-      includedFieldsVertices: Array[Int] = null,
-      lineDelimiterEdges: String = "\n",
-      fieldDelimiterEdges: String = ",",
-      quoteCharacterEdges: Character = null,
-      ignoreFirstLineEdges: Boolean = false,
-      ignoreCommentsEdges: String = null,
-      lenientEdges: Boolean = false,
-      includedFieldsEdges: Array[Int] = null,
-      mapper: MapFunction[K, VV] = null) = {
-
-    // with vertex and edge values
-    if (readVertices && hasEdgeValues) {
-      if (pathVertices.equals(null)) {
-        throw new IllegalArgumentException(
-            "The vertices file path must be specified when readVertices is true.")
-      } else {
-        val vertices = env.readCsvFile[(K, VV)](pathVertices, lineDelimiterVertices,
-            fieldDelimiterVertices, quoteCharacterVertices, ignoreFirstLineVertices,
-            ignoreCommentsVertices, lenientVertices, includedFieldsVertices)
-
-        val edges = env.readCsvFile[(K, K, EV)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
-            quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
-            includedFieldsEdges)
-     
-        fromTupleDataSet[K, VV, EV](vertices, edges, env) 
-      }
-    }
-    // with vertex value and no edge value
-    else if (readVertices && (!hasEdgeValues)) {
-       if (pathVertices.equals(null)) {
-        throw new IllegalArgumentException(
-            "The vertices file path must be specified when readVertices is true.")
-      } else {
-        val vertices = env.readCsvFile[(K, VV)](pathVertices, lineDelimiterVertices,
-            fieldDelimiterVertices, quoteCharacterVertices, ignoreFirstLineVertices,
-            ignoreCommentsVertices, lenientVertices, includedFieldsVertices)
-
-        val edges = env.readCsvFile[(K, K)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
-            quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
-            includedFieldsEdges).map(edge => (edge._1, edge._2, NullValue.getInstance))
-
-        fromTupleDataSet[K, VV, NullValue](vertices, edges, env)
-      }
-    }
-    // with edge value and no vertex value
-    else if ((!readVertices) && hasEdgeValues) {
-      val edges = env.readCsvFile[(K, K, EV)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
-        quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
-        includedFieldsEdges)
-
-      // initializer provided
-      if (mapper != null) {
-        fromTupleDataSet[K, VV, EV](edges, mapper, env)
-      }
-      else {
-        fromTupleDataSet[K, EV](edges, env) 
-      }
-    }
-    // with no edge value and no vertex value
-    else {
-      val edges = env.readCsvFile[(K, K)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
-      quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges,
-      lenientEdges, includedFieldsEdges).map(edge => (edge._1, edge._2, NullValue.getInstance))
-
-      // no initializer provided
-      if (mapper != null) {
-        fromTupleDataSet[K, VV, NullValue](edges, mapper, env)
-      }
-      else {
-        fromTupleDataSet[K, NullValue](edges, env) 
-      }
-    }
-  }
-// scalastyle:on
-
-}
-
-/**
- * Represents a graph consisting of {@link Edge edges} and {@link Vertex vertices}.
- * @param jgraph the underlying java api Graph.
- * @tparam K the key type for vertex and edge identifiers
- * @tparam VV the value type for vertices
- * @tparam EV the value type for edges
- * @see org.apache.flink.graph.Edge
- * @see org.apache.flink.graph.Vertex
- */
-final class Graph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
-TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) {
-
-  private[flink] def getWrappedGraph = jgraph
-
-
-  private[flink] def clean[F <: AnyRef](f: F, checkSerializable: Boolean = true): F = {
-    if (jgraph.getContext.getConfig.isClosureCleanerEnabled) {
-      ClosureCleaner.clean(f, checkSerializable)
-    }
-    ClosureCleaner.ensureSerializable(f)
-    f
-  }
-
-  /**
-   * @return the vertex DataSet.
-   */
-  def getVertices = wrap(jgraph.getVertices)
-
-  /**
-   * @return the edge DataSet.
-   */
-  def getEdges = wrap(jgraph.getEdges)
-
-  /**
-   * @return the vertex DataSet as Tuple2.
-   */
-  def getVerticesAsTuple2(): DataSet[(K, VV)] = {
-    wrap(jgraph.getVerticesAsTuple2).map(jtuple => (jtuple.f0, jtuple.f1))
-  }
-
-  /**
-   * @return the edge DataSet as Tuple3.
-   */
-  def getEdgesAsTuple3(): DataSet[(K, K, EV)] = {
-    wrap(jgraph.getEdgesAsTuple3).map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2))
-  }
-
-  /**
-  * @return a DataSet of Triplets,
-  * consisting of (srcVertexId, trgVertexId, srcVertexValue, trgVertexValue, edgeValue)
-  */
-  def getTriplets(): DataSet[Triplet[K, VV, EV]] = {
-    wrap(jgraph.getTriplets())
-  }
-
-  /**
-   * Apply a function to the attribute of each vertex in the graph.
-   *
-   * @param mapper the map function to apply.
-   * @return a new graph
-   */
-  def mapVertices[NV: TypeInformation : ClassTag](mapper: MapFunction[Vertex[K, VV], NV]):
-  Graph[K, NV, EV] = {
-    new Graph[K, NV, EV](jgraph.mapVertices[NV](
-      mapper,
-      createTypeInformation[Vertex[K, NV]]
-    ))
-  }
-
-  /**
-   * Apply a function to the attribute of each vertex in the graph.
-   *
-   * @param fun the map function to apply.
-   * @return a new graph
-   */
-  def mapVertices[NV: TypeInformation : ClassTag](fun: Vertex[K, VV] => NV): Graph[K, NV, EV] = {
-    val mapper: MapFunction[Vertex[K, VV], NV] = new MapFunction[Vertex[K, VV], NV] {
-      val cleanFun = clean(fun)
-
-      def map(in: Vertex[K, VV]): NV = cleanFun(in)
-    }
-    new Graph[K, NV, EV](jgraph.mapVertices[NV](mapper, createTypeInformation[Vertex[K, NV]]))
-  }
-
-  /**
-   * Apply a function to the attribute of each edge in the graph.
-   *
-   * @param mapper the map function to apply.
-   * @return a new graph
-   */
-  def mapEdges[NV: TypeInformation : ClassTag](mapper: MapFunction[Edge[K, EV], NV]): Graph[K,
-    VV, NV] = {
-    new Graph[K, VV, NV](jgraph.mapEdges[NV](
-      mapper,
-      createTypeInformation[Edge[K, NV]]
-    ))
-  }
-
-  /**
-   * Apply a function to the attribute of each edge in the graph.
-   *
-   * @param fun the map function to apply.
-   * @return a new graph
-   */
-  def mapEdges[NV: TypeInformation : ClassTag](fun: Edge[K, EV] => NV): Graph[K, VV, NV] = {
-    val mapper: MapFunction[Edge[K, EV], NV] = new MapFunction[Edge[K, EV], NV] {
-      val cleanFun = clean(fun)
-
-      def map(in: Edge[K, EV]): NV = cleanFun(in)
-    }
-    new Graph[K, VV, NV](jgraph.mapEdges[NV](mapper, createTypeInformation[Edge[K, NV]]))
-  }
-
-  /**
-   * Joins the vertex DataSet of this graph with an input DataSet and applies
-   * a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @return a new graph where the vertex values have been updated.
-   */
-  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: MapFunction[
-    (VV, T), VV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
-      override def map(value: jtuple.Tuple2[VV, T]): VV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the vertex DataSet of this graph with an input DataSet and applies
-   * a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @return a new graph where the vertex values have been updated.
-   */
-  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (VV, T) => VV):
-  Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
-      val cleanFun = clean(fun)
-
-      override def map(value: jtuple.Tuple2[VV, T]): VV = {
-        cleanFun(value.f0, value.f1)
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on a composite key of both
-   * source and target and applies a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], mapper: MapFunction[
-    (EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
-      scalatuple._2, scalatuple._3)).javaSet
-    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on a composite key of both
-   * source and target and applies a UDF on the resulted values.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], fun: (EV, T) => EV):
-  Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      val cleanFun = clean(fun)
-
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
-      scalatuple._2, scalatuple._3)).javaSet
-    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on the source key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. In case the inputDataSet contains the same key more
-   * than once, only the first value will be considered.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
-  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on the source key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. In case the inputDataSet contains the same key more
-   * than once, only the first value will be considered.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
-    EV): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      val cleanFun = clean(fun)
-
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on the target key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. Should the inputDataSet contain the same key more
-   * than once, only the first value will be considered.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param mapper the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
-  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        mapper.map((value.f0, value.f1))
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Joins the edge DataSet with an input DataSet on the target key of the
-   * edges and the first attribute of the input DataSet and applies a UDF on
-   * the resulted values. Should the inputDataSet contain the same key more
-   * than once, only the first value will be considered.
-   *
-   * @param inputDataSet the DataSet to join with.
-   * @param fun the UDF map function to apply.
-   * @tparam T the return type
-   * @return a new graph where the edge values have been updated.
-   */
-  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
-    EV): Graph[K, VV, EV] = {
-    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
-      val cleanFun = clean(fun)
-
-      override def map(value: jtuple.Tuple2[EV, T]): EV = {
-        cleanFun(value.f0, value.f1)
-      }
-    }
-    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
-      scalatuple._2)).javaSet
-    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
-  }
-
-  /**
-   * Apply filtering functions to the graph and return a sub-graph that
-   * satisfies the predicates for both vertices and edges.
-   *
-   * @param vertexFilter the filter function for vertices.
-   * @param edgeFilter the filter function for edges.
-   * @return the resulting sub-graph.
-   */
-  def subgraph(vertexFilter: FilterFunction[Vertex[K, VV]], edgeFilter: FilterFunction[Edge[K,
-    EV]]) = {
-    wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter))
-  }
-
-  /**
-   * Apply filtering functions to the graph and return a sub-graph that
-   * satisfies the predicates for both vertices and edges.
-   *
-   * @param vertexFilterFun the filter function for vertices.
-   * @param edgeFilterFun the filter function for edges.
-   * @return the resulting sub-graph.
-   */
-  def subgraph(vertexFilterFun: Vertex[K, VV] => Boolean, edgeFilterFun: Edge[K, EV] =>
-    Boolean) = {
-    val vertexFilter = new FilterFunction[Vertex[K, VV]] {
-      val cleanVertexFun = clean(vertexFilterFun)
-
-      override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value)
-    }
-
-    val edgeFilter = new FilterFunction[Edge[K, EV]] {
-      val cleanEdgeFun = clean(edgeFilterFun)
-
-      override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value)
-    }
-
-    wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter))
-  }
-
-  /**
-   * Apply a filtering function to the graph and return a sub-graph that
-   * satisfies the predicates only for the vertices.
-   *
-   * @param vertexFilter the filter function for vertices.
-   * @return the resulting sub-graph.
-   */
-  def filterOnVertices(vertexFilter: FilterFunction[Vertex[K, VV]]) = {
-    wrapGraph(jgraph.filterOnVertices(vertexFilter))
-  }
-
-  /**
-   * Apply a filtering function to the graph and return a sub-graph that
-   * satisfies the predicates only for the vertices.
-   *
-   * @param vertexFilterFun the filter function for vertices.
-   * @return the resulting sub-graph.
-   */
-  def filterOnVertices(vertexFilterFun: Vertex[K, VV] => Boolean) = {
-    val vertexFilter = new FilterFunction[Vertex[K, VV]] {
-      val cleanVertexFun = clean(vertexFilterFun)
-
-      override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value)
-    }
-
-    wrapGraph(jgraph.filterOnVertices(vertexFilter))
-  }
-
-  /**
-   * Apply a filtering function to the graph and return a sub-graph that
-   * satisfies the predicates only for the edges.
-   *
-   * @param edgeFilter the filter function for edges.
-   * @return the resulting sub-graph.
-   */
-  def filterOnEdges(edgeFilter: FilterFunction[Edge[K, EV]]) = {
-    wrapGraph(jgraph.filterOnEdges(edgeFilter))
-  }
-
-  /**
-   * Apply a filtering function to the graph and return a sub-graph that
-   * satisfies the predicates only for the edges.
-   *
-   * @param edgeFilterFun the filter function for edges.
-   * @return the resulting sub-graph.
-   */
-  def filterOnEdges(edgeFilterFun: Edge[K, EV] => Boolean) = {
-    val edgeFilter = new FilterFunction[Edge[K, EV]] {
-      val cleanEdgeFun = clean(edgeFilterFun)
-
-      override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value)
-    }
-
-    wrapGraph(jgraph.filterOnEdges(edgeFilter))
-  }
-
-  /**
-   * Return the in-degree of all vertices in the graph
-   *
-   * @return A DataSet of Tuple2<vertexId, inDegree>
-   */
-  def inDegrees(): DataSet[(K, Long)] = {
-    wrap(jgraph.inDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
-  }
-
-  /**
-   * Return the out-degree of all vertices in the graph
-   *
-   * @return A DataSet of Tuple2<vertexId, outDegree>
-   */
-  def outDegrees(): DataSet[(K, Long)] = {
-    wrap(jgraph.outDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
-  }
-
-  /**
-   * Return the degree of all vertices in the graph
-   *
-   * @return A DataSet of Tuple2<vertexId, degree>
-   */
-  def getDegrees(): DataSet[(K, Long)] = {
-    wrap(jgraph.getDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
-  }
-
-  /**
-   * This operation adds all inverse-direction edges to the graph.
-   *
-   * @return the undirected graph.
-   */
-  def getUndirected(): Graph[K, VV, EV] = {
-    new Graph(jgraph.getUndirected)
-  }
-
-  /**
-   * Reverse the direction of the edges in the graph
-   *
-   * @return a new graph with all edges reversed
-   * @throws UnsupportedOperationException
-   */
-  def reverse(): Graph[K, VV, EV] = {
-    new Graph(jgraph.reverse())
-  }
-
-  /**
-   * Compute an aggregate over the edges of each vertex. The function applied
-   * on the edges has access to the vertex value.
-   *
-   * @param edgesFunction the function to apply to the neighborhood
-   * @param direction     the edge direction (in-, out-, all-)
-   * @tparam T           the output type
-   * @return a dataset of a T
-   */
-  def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction:
-                                                        EdgesFunctionWithVertexValue[K, VV, EV,
-                                                          T], direction: EdgeDirection):
-  DataSet[T] = {
-    wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T]))
-  }
-
-  /**
-   * Compute an aggregate over the edges of each vertex. The function applied
-   * on the edges has access to the vertex value.
-   *
-   * @param edgesFunction the function to apply to the neighborhood
-   * @param direction     the edge direction (in-, out-, all-)
-   * @tparam T           the output type
-   * @return a dataset of a T
-   */
-  def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction: EdgesFunction[K, EV, T],
-                                                        direction: EdgeDirection): DataSet[T] = {
-    wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T]))
-  }
-
-  /**
-   * Compute an aggregate over the neighbors (edges and vertices) of each
-   * vertex. The function applied on the neighbors has access to the vertex
-   * value.
-   *
-   * @param neighborsFunction the function to apply to the neighborhood
-   * @param direction         the edge direction (in-, out-, all-)
-   * @tparam T               the output type
-   * @return a dataset of a T
-   */
-  def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction:
-                                                            NeighborsFunctionWithVertexValue[K,
-                                                              VV, EV, T], direction:
-                                                            EdgeDirection): DataSet[T] = {
-    wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T]))
-  }
-
-  /**
-   * Compute an aggregate over the neighbors (edges and vertices) of each
-   * vertex.
-   *
-   * @param neighborsFunction the function to apply to the neighborhood
-   * @param direction         the edge direction (in-, out-, all-)
-   * @tparam T               the output type
-   * @return a dataset of a T
-   */
-  def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction:
-                                                            NeighborsFunction[K, VV, EV, T],
-                                                            direction: EdgeDirection):
-  DataSet[T] = {
-    wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T]))
-  }
-
-  /**
-   * @return a long integer representing the number of vertices
-   */
-  def numberOfVertices(): Long = {
-    jgraph.numberOfVertices()
-  }
-
-  /**
-   * @return a long integer representing the number of edges
-   */
-  def numberOfEdges(): Long = {
-    jgraph.numberOfEdges()
-  }
-
-  /**
-   * @return The IDs of the vertices as DataSet
-   */
-  def getVertexIds(): DataSet[K] = {
-    wrap(jgraph.getVertexIds)
-  }
-
-  /**
-   * @return The IDs of the edges as DataSet
-   */
-  def getEdgeIds(): DataSet[(K, K)] = {
-    wrap(jgraph.getEdgeIds).map(jtuple => (jtuple.f0, jtuple.f1))
-  }
-
-  /**
-   * Adds the input vertex to the graph. If the vertex already
-   * exists in the graph, it will not be added again.
-   *
-   * @param vertex the vertex to be added
-   * @return the new graph containing the existing vertices as well as the one just added
-   */
-  def addVertex(vertex: Vertex[K, VV]) = {
-    wrapGraph(jgraph.addVertex(vertex))
-  }
-
-  /**
-  * Adds the list of vertices, passed as input, to the graph.
-  * If the vertices already exist in the graph, they will not be added once more.
-  *
-  * @param verticesToAdd the list of vertices to add
-  * @return the new graph containing the existing and newly added vertices
-  */
-  def addVertices(vertices: List[Vertex[K, VV]]): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.addVertices(vertices.asJava))
-  }
-
-  /**
-  * Adds the given list edges to the graph.
-  *
-  * When adding an edge for a non-existing set of vertices,
-  * the edge is considered invalid and ignored.
-  *
-  * @param newEdges the data set of edges to be added
-  * @return a new graph containing the existing edges plus the newly added edges.
-  */
-  def addEdges(edges: List[Edge[K, EV]]): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.addEdges(edges.asJava))
-  }
-
-    /**
-   * Adds the given edge to the graph. If the source and target vertices do
-   * not exist in the graph, they will also be added.
-   *
-   * @param source the source vertex of the edge
-   * @param target the target vertex of the edge
-   * @param edgeValue the edge value
-   * @return the new graph containing the existing vertices and edges plus the
-   *         newly added edge
-   */
-  def addEdge(source: Vertex[K, VV], target: Vertex[K, VV], edgeValue: EV) = {
-    wrapGraph(jgraph.addEdge(source, target, edgeValue))
-  }
-
-  /**
-   * Removes the given vertex and its edges from the graph.
-   *
-   * @param vertex the vertex to remove
-   * @return the new graph containing the existing vertices and edges without
-   *         the removed vertex and its edges
-   */
-  def removeVertex(vertex: Vertex[K, VV]) = {
-    wrapGraph(jgraph.removeVertex(vertex))
-  }
-
-    /**
-   * Removes the given vertex and its edges from the graph.
-   *
-   * @param vertex the vertex to remove
-   * @return the new graph containing the existing vertices and edges without
-   *         the removed vertex and its edges
-   */
-  def removeVertices(vertices: List[Vertex[K, VV]]): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.removeVertices(vertices.asJava))
-  }
-
-  /**
-   * Removes all edges that match the given edge from the graph.
-   *
-   * @param edge the edge to remove
-   * @return the new graph containing the existing vertices and edges without
-   *         the removed edges
-   */
-  def removeEdge(edge: Edge[K, EV]) = {
-    wrapGraph(jgraph.removeEdge(edge))
-  }
-
-  /**
-   * Removes all the edges that match the edges in the given data set from the graph.
-   *
-   * @param edgesToBeRemoved the list of edges to be removed
-   * @return a new graph where the edges have been removed and in which the vertices remained intact
-   */
-  def removeEdges(edges: List[Edge[K, EV]]): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.removeEdges(edges.asJava))
-  }
-
-  /**
-   * Performs union on the vertices and edges sets of the input graphs
-   * removing duplicate vertices but maintaining duplicate edges.
-   *
-   * @param graph the graph to perform union with
-   * @return a new graph
-   */
-  def union(graph: Graph[K, VV, EV]) = {
-    wrapGraph(jgraph.union(graph.getWrappedGraph))
-  }
-
-  /**
-  * Performs Difference on the vertex and edge sets of the input graphs
-  * removes common vertices and edges. If a source/target vertex is removed,
-  * its corresponding edge will also be removed
-  * @param graph the graph to perform difference with
-  * @return a new graph where the common vertices and edges have been removed
-  */
-  def difference(graph: Graph[K, VV, EV]) = {
-    wrapGraph(jgraph.difference(graph.getWrappedGraph))
-  }
-
-  /**
-   * Compute an aggregate over the neighbor values of each
-   * vertex.
-   *
-   * @param reduceNeighborsFunction the function to apply to the neighborhood
-   * @param direction               the edge direction (in-, out-, all-)
-   * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
-   */
-  def reduceOnNeighbors(reduceNeighborsFunction: ReduceNeighborsFunction[VV], direction:
-  EdgeDirection): DataSet[(K, VV)] = {
-    wrap(jgraph.reduceOnNeighbors(reduceNeighborsFunction, direction)).map(jtuple => (jtuple
-      .f0, jtuple.f1))
-  }
-
-  /**
-   * Compute an aggregate over the edge values of each vertex.
-   *
-   * @param reduceEdgesFunction the function to apply to the neighborhood
-   * @param direction           the edge direction (in-, out-, all-)
-   * @return a Dataset containing one value per vertex(vertex key, aggegate edge value)
-   * @throws IllegalArgumentException
-   */
-  def reduceOnEdges(reduceEdgesFunction: ReduceEdgesFunction[EV], direction: EdgeDirection):
-  DataSet[(K, EV)] = {
-    wrap(jgraph.reduceOnEdges(reduceEdgesFunction, direction)).map(jtuple => (jtuple.f0,
-      jtuple.f1))
-  }
-
-  def run[T: TypeInformation : ClassTag](algorithm: GraphAlgorithm[K, VV, EV, T]):
-  T = {
-    jgraph.run(algorithm)
-  }
-
-  /**
-   * Runs a Vertex-Centric iteration on the graph.
-   * No configuration options are provided.
-   *
-   * @param vertexUpdateFunction the vertex update function
-   * @param messagingFunction the messaging function
-   * @param maxIterations maximum number of iterations to perform
-   *
-   * @return the updated Graph after the vertex-centric iteration has converged or
-   *         after maximumNumberOfIterations.
-   */
-  def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M],
-                                   messagingFunction: MessagingFunction[K, VV, M, EV],
-                                   maxIterations: Int): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
-      maxIterations))
-  }
-
-  /**
-   * Runs a Vertex-Centric iteration on the graph with configuration options.
-   *
-   * @param vertexUpdateFunction the vertex update function
-   * @param messagingFunction the messaging function
-   * @param maxIterations maximum number of iterations to perform
-   * @param parameters the iteration configuration parameters
-   *
-   * @return the updated Graph after the vertex-centric iteration has converged or
-   *         after maximumNumberOfIterations.
-   */
-  def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M],
-                                   messagingFunction: MessagingFunction[K, VV, M, EV],
-                                   maxIterations: Int, parameters: VertexCentricConfiguration):
-  Graph[K, VV, EV] = {
-    wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
-      maxIterations, parameters))
-  }
-
-  /**
-   * Runs a Gather-Sum-Apply iteration on the graph.
-   * No configuration options are provided.
-   *
-   * @param gatherFunction the gather function collects information about adjacent
-   *                       vertices and edges
-   * @param sumFunction the sum function aggregates the gathered information
-   * @param applyFunction the apply function updates the vertex values with the aggregates
-   * @param maxIterations maximum number of iterations to perform
-   * @tparam M the intermediate type used between gather, sum and apply
-   *
-   * @return the updated Graph after the gather-sum-apply iteration has converged or
-   *         after maximumNumberOfIterations.
-   */
-  def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction:
-  SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int): Graph[K,
-    VV, EV] = {
-    wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
-      maxIterations))
-  }
-
-  /**
-   * Runs a Gather-Sum-Apply iteration on the graph with configuration options.
-   *
-   * @param gatherFunction the gather function collects information about adjacent
-   *                       vertices and edges
-   * @param sumFunction the sum function aggregates the gathered information
-   * @param applyFunction the apply function updates the vertex values with the aggregates
-   * @param maxIterations maximum number of iterations to perform
-   * @param parameters the iteration configuration parameters
-   * @tparam M the intermediate type used between gather, sum and apply
-   *
-   * @return the updated Graph after the gather-sum-apply iteration has converged or
-   *         after maximumNumberOfIterations.
-   */
-  def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction:
-  SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int,
-                                    parameters: GSAConfiguration): Graph[K, VV, EV] = {
-    wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
-      maxIterations, parameters))
-  }
-
-  def validate(validator: GraphValidator[K, VV, EV]): Boolean = {
-    jgraph.validate(validator)
-  }
-
-}


[17/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
new file mode 100644
index 0000000..afd4ffd
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricConfiguration.java
@@ -0,0 +1,135 @@
+/*
+ * 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.spargel;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.IterationConfiguration;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A VertexCentricConfiguration object can be used to set the iteration name and
+ * degree of parallelism, to register aggregators and use broadcast sets in
+ * the {@link org.apache.flink.graph.spargel.VertexUpdateFunction} and {@link org.apache.flink.graph.spargel.MessagingFunction}
+ *
+ * The VertexCentricConfiguration object is passed as an argument to
+ * {@link org.apache.flink.graph.Graph#runVertexCentricIteration (
+ * org.apache.flink.graph.spargel.VertexUpdateFunction, org.apache.flink.graph.spargel.MessagingFunction, int,
+ * VertexCentricConfiguration)}.
+ */
+public class VertexCentricConfiguration extends IterationConfiguration {
+
+	/** the broadcast variables for the update function **/
+	private List<Tuple2<String, DataSet<?>>> bcVarsUpdate = new ArrayList<Tuple2<String,DataSet<?>>>();
+
+	/** the broadcast variables for the messaging function **/
+	private List<Tuple2<String, DataSet<?>>> bcVarsMessaging = new ArrayList<Tuple2<String,DataSet<?>>>();
+
+	/** flag that defines whether the degrees option is set **/
+	private boolean optDegrees = false;
+
+	/** the direction in which the messages should be sent **/
+	private EdgeDirection direction = EdgeDirection.OUT;
+
+	public VertexCentricConfiguration() {}
+
+	/**
+	 * Adds a data set as a broadcast set to the messaging function.
+	 *
+	 * @param name The name under which the broadcast data is available in the messaging function.
+	 * @param data The data set to be broadcasted.
+	 */
+	public void addBroadcastSetForMessagingFunction(String name, DataSet<?> data) {
+		this.bcVarsMessaging.add(new Tuple2<String, DataSet<?>>(name, data));
+	}
+
+	/**
+	 * Adds a data set as a broadcast set to the vertex update function.
+	 *
+	 * @param name The name under which the broadcast data is available in the vertex update function.
+	 * @param data The data set to be broadcasted.
+	 */
+	public void addBroadcastSetForUpdateFunction(String name, DataSet<?> data) {
+		this.bcVarsUpdate.add(new Tuple2<String, DataSet<?>>(name, data));
+	}
+
+	/**
+	 * Get the broadcast variables of the VertexUpdateFunction.
+	 *
+	 * @return a List of Tuple2, where the first field is the broadcast variable name
+	 * and the second field is the broadcast DataSet.
+	 */
+	public List<Tuple2<String, DataSet<?>>> getUpdateBcastVars() {
+		return this.bcVarsUpdate;
+	}
+
+	/**
+	 * Get the broadcast variables of the MessagingFunction.
+	 *
+	 * @return a List of Tuple2, where the first field is the broadcast variable name
+	 * and the second field is the broadcast DataSet.
+	 */
+	public List<Tuple2<String, DataSet<?>>> getMessagingBcastVars() {
+		return this.bcVarsMessaging;
+	}
+
+	/**
+	 * Gets whether the degrees option is set.
+	 * By default, the degrees option is not set.
+	 *
+	 * @return True, if the degree option is set, false otherwise.
+	 */
+	public boolean isOptDegrees() {
+		return optDegrees;
+	}
+
+	/**
+	 * Sets the degree option.
+	 * By default, the degrees option is not set.
+	 *
+	 * @param optDegrees True, to set this option, false otherwise.
+	 */
+	public void setOptDegrees(boolean optDegrees) {
+		this.optDegrees = optDegrees;
+	}
+
+	/**
+	 * Gets the direction in which messages are sent in the MessagingFunction.
+	 * By default the messaging direction is OUT.
+	 *
+	 * @return an EdgeDirection, which can be either IN, OUT or ALL.
+	 */
+	public EdgeDirection getDirection() {
+		return direction;
+	}
+
+	/**
+	 * Sets the direction in which messages are sent in the MessagingFunction.
+	 * By default the messaging direction is OUT.
+	 *
+	 * @param direction - IN, OUT or ALL
+	 */
+	public void setDirection(EdgeDirection direction) {
+		this.direction = direction;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
new file mode 100644
index 0000000..b3a470e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
@@ -0,0 +1,686 @@
+/*
+ * 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.spargel;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.operators.CoGroupOperator;
+import org.apache.flink.api.java.operators.CustomUnaryOperation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.util.Collector;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * This class represents iterative graph computations, programmed in a vertex-centric perspective.
+ * It is a special case of <i>Bulk Synchronous Parallel<i> computation. The paradigm has also been
+ * implemented by Google's <i>Pregel</i> system and by <i>Apache Giraph</i>.
+ * <p>
+ * Vertex centric algorithms operate on graphs, which are defined through vertices and edges. The 
+ * algorithms send messages along the edges and update the state of vertices based on
+ * the old state and the incoming messages. All vertices have an initial state.
+ * The computation terminates once no vertex updates it state any more.
+ * Additionally, a maximum number of iterations (supersteps) may be specified.
+ * <p>
+ * The computation is here represented by two functions:
+ * <ul>
+ *   <li>The {@link VertexUpdateFunction} receives incoming messages and may updates the state for
+ *   the vertex. If a state is updated, messages are sent from this vertex. Initially, all vertices are
+ *   considered updated.</li>
+ *   <li>The {@link MessagingFunction} takes the new vertex state and sends messages along the outgoing
+ *   edges of the vertex. The outgoing edges may optionally have an associated value, such as a weight.</li>
+ * </ul>
+ * <p>
+ *
+ * Vertex-centric graph iterations are are run by calling
+ * {@link Graph#runVertexCentricIteration(VertexUpdateFunction, MessagingFunction, int)}.
+ *
+ * @param <K> The type of the vertex key (the vertex identifier).
+ * @param <VV> The type of the vertex value (the state of the vertex).
+ * @param <Message> The type of the message sent between vertices along the edges.
+ * @param <EV> The type of the values that are associated with the edges.
+ */
+public class VertexCentricIteration<K, VV, Message, EV> 
+	implements CustomUnaryOperation<Vertex<K, VV>, Vertex<K, VV>>
+{
+	private final VertexUpdateFunction<K, VV, Message> updateFunction;
+
+	private final MessagingFunction<K, VV, Message, EV> messagingFunction;
+	
+	private final DataSet<Edge<K, EV>> edgesWithValue;
+	
+	private final int maximumNumberOfIterations;
+	
+	private final TypeInformation<Message> messageType;
+	
+	private DataSet<Vertex<K, VV>> initialVertices;
+
+	private VertexCentricConfiguration configuration;
+
+	// ----------------------------------------------------------------------------------
+	
+	private VertexCentricIteration(VertexUpdateFunction<K, VV, Message> uf,
+			MessagingFunction<K, VV, Message, EV> mf,
+			DataSet<Edge<K, EV>> edgesWithValue, 
+			int maximumNumberOfIterations)
+	{
+		Preconditions.checkNotNull(uf);
+		Preconditions.checkNotNull(mf);
+		Preconditions.checkNotNull(edgesWithValue);
+		Preconditions.checkArgument(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one.");
+
+		this.updateFunction = uf;
+		this.messagingFunction = mf;
+		this.edgesWithValue = edgesWithValue;
+		this.maximumNumberOfIterations = maximumNumberOfIterations;		
+		this.messageType = getMessageType(mf);
+	}
+	
+	private TypeInformation<Message> getMessageType(MessagingFunction<K, VV, Message, EV> mf) {
+		return TypeExtractor.createTypeInfo(MessagingFunction.class, mf.getClass(), 2, null, null);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Custom Operator behavior
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Sets the input data set for this operator. In the case of this operator this input data set represents
+	 * the set of vertices with their initial state.
+	 * 
+	 * @param inputData The input data set, which in the case of this operator represents the set of
+	 *                  vertices with their initial state.
+	 * 
+	 * @see org.apache.flink.api.java.operators.CustomUnaryOperation#setInput(org.apache.flink.api.java.DataSet)
+	 */
+	@Override
+	public void setInput(DataSet<Vertex<K, VV>> inputData) {
+		this.initialVertices = inputData;
+	}
+	
+	/**
+	 * Creates the operator that represents this vertex-centric graph computation.
+	 * 
+	 * @return The operator that represents this vertex-centric graph computation.
+	 */
+	@Override
+	public DataSet<Vertex<K, VV>> createResult() {
+		if (this.initialVertices == null) {
+			throw new IllegalStateException("The input data set has not been set.");
+		}
+
+		// prepare some type information
+		TypeInformation<K> keyType = ((TupleTypeInfo<?>) initialVertices.getType()).getTypeAt(0);
+		TypeInformation<Tuple2<K, Message>> messageTypeInfo = new TupleTypeInfo<Tuple2<K,Message>>(keyType, messageType);
+
+		// create a graph
+		Graph<K, VV, EV> graph =
+				Graph.fromDataSet(initialVertices, edgesWithValue, initialVertices.getExecutionEnvironment());
+
+		// check whether the numVertices option is set and, if so, compute the total number of vertices
+		// and set it within the messaging and update functions
+
+		if (this.configuration != null && this.configuration.isOptNumVertices()) {
+			try {
+				long numberOfVertices = graph.numberOfVertices();
+				messagingFunction.setNumberOfVertices(numberOfVertices);
+				updateFunction.setNumberOfVertices(numberOfVertices);
+			} catch (Exception e) {
+				e.printStackTrace();
+			}
+		}
+
+		if(this.configuration != null) {
+			messagingFunction.setDirection(this.configuration.getDirection());
+		} else {
+			messagingFunction.setDirection(EdgeDirection.OUT);
+		}
+
+		// retrieve the direction in which the updates are made and in which the messages are sent
+		EdgeDirection messagingDirection = messagingFunction.getDirection();
+
+		// check whether the degrees option is set and, if so, compute the in and the out degrees and
+		// add them to the vertex value
+		if(this.configuration != null && this.configuration.isOptDegrees()) {
+			return createResultVerticesWithDegrees(graph, messagingDirection, messageTypeInfo);
+		} else {
+			return createResultSimpleVertex(messagingDirection, messageTypeInfo);
+		}
+	}
+
+	/**
+	 * Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as
+	 * a weight or distance).
+	 * 
+	 * @param edgesWithValue The data set containing edges.
+	 * @param uf The function that updates the state of the vertices from the incoming messages.
+	 * @param mf The function that turns changed vertex states into messages along the edges.
+	 * 
+	 * @param <K> The type of the vertex key (the vertex identifier).
+	 * @param <VV> The type of the vertex value (the state of the vertex).
+	 * @param <Message> The type of the message sent between vertices along the edges.
+	 * @param <EV> The type of the values that are associated with the edges.
+	 * 
+	 * @return An in stance of the vertex-centric graph computation operator.
+	 */
+	public static final <K, VV, Message, EV>
+			VertexCentricIteration<K, VV, Message, EV> withEdges(
+					DataSet<Edge<K, EV>> edgesWithValue,
+					VertexUpdateFunction<K, VV, Message> uf,
+					MessagingFunction<K, VV, Message, EV> mf,
+					int maximumNumberOfIterations)
+	{
+		return new VertexCentricIteration<K, VV, Message, EV>(uf, mf, edgesWithValue, maximumNumberOfIterations);
+	}
+
+	/**
+	 * Configures this vertex-centric iteration with the provided parameters.
+	 *
+	 * @param parameters the configuration parameters
+	 */
+	public void configure(VertexCentricConfiguration parameters) {
+		this.configuration = parameters;
+	}
+
+	/**
+	 * @return the configuration parameters of this vertex-centric iteration
+	 */
+	public VertexCentricConfiguration getIterationConfiguration() {
+		return this.configuration;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Wrapping UDFs
+	// --------------------------------------------------------------------------------------------
+
+	private static abstract class VertexUpdateUdf<K, VVWithDegrees, Message> extends RichCoGroupFunction<
+		Tuple2<K, Message>, Vertex<K, VVWithDegrees>, Vertex<K, VVWithDegrees>>
+		implements ResultTypeQueryable<Vertex<K, VVWithDegrees>>
+	{
+		private static final long serialVersionUID = 1L;
+		
+		final VertexUpdateFunction<K, VVWithDegrees, Message> vertexUpdateFunction;
+
+		final MessageIterator<Message> messageIter = new MessageIterator<Message>();
+		
+		private transient TypeInformation<Vertex<K, VVWithDegrees>> resultType;
+		
+		
+		private VertexUpdateUdf(VertexUpdateFunction<K, VVWithDegrees, Message> vertexUpdateFunction,
+				TypeInformation<Vertex<K, VVWithDegrees>> resultType)
+		{
+			this.vertexUpdateFunction = vertexUpdateFunction;
+			this.resultType = resultType;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
+				this.vertexUpdateFunction.init(getIterationRuntimeContext());
+			}
+			this.vertexUpdateFunction.preSuperstep();
+		}
+		
+		@Override
+		public void close() throws Exception {
+			this.vertexUpdateFunction.postSuperstep();
+		}
+
+		@Override
+		public TypeInformation<Vertex<K, VVWithDegrees>> getProducedType() {
+			return this.resultType;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class VertexUpdateUdfSimpleVV<K, VV, Message> extends VertexUpdateUdf<K, VV, Message> {
+
+		private VertexUpdateUdfSimpleVV(VertexUpdateFunction<K, VV, Message> vertexUpdateFunction, TypeInformation<Vertex<K, VV>> resultType) {
+			super(vertexUpdateFunction, resultType);
+		}
+
+		@Override
+		public void coGroup(Iterable<Tuple2<K, Message>> messages,
+							Iterable<Vertex<K, VV>> vertex,
+							Collector<Vertex<K, VV>> out) throws Exception {
+			final Iterator<Vertex<K, VV>> vertexIter = vertex.iterator();
+
+			if (vertexIter.hasNext()) {
+				Vertex<K, VV> vertexState = vertexIter.next();
+
+				@SuppressWarnings("unchecked")
+				Iterator<Tuple2<?, Message>> downcastIter = (Iterator<Tuple2<?, Message>>) (Iterator<?>) messages.iterator();
+				messageIter.setSource(downcastIter);
+
+				vertexUpdateFunction.setOutput(vertexState, out);
+				vertexUpdateFunction.updateVertex(vertexState, messageIter);
+			}
+			else {
+				final Iterator<Tuple2<K, Message>> messageIter = messages.iterator();
+				if (messageIter.hasNext()) {
+					String message = "Target vertex does not exist!.";
+					try {
+						Tuple2<K, Message> next = messageIter.next();
+						message = "Target vertex '" + next.f0 + "' does not exist!.";
+					} catch (Throwable t) {}
+					throw new Exception(message);
+				} else {
+					throw new Exception();
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class VertexUpdateUdfVVWithDegrees<K, VV, Message> extends VertexUpdateUdf<K, Tuple3<VV, Long, Long>, Message> {
+
+		private VertexUpdateUdfVVWithDegrees(VertexUpdateFunction<K, Tuple3<VV, Long, Long>, Message> vertexUpdateFunction,
+				TypeInformation<Vertex<K, Tuple3<VV, Long, Long>>> resultType) {
+			super(vertexUpdateFunction, resultType);
+		}
+		
+		@Override
+		public void coGroup(Iterable<Tuple2<K, Message>> messages, Iterable<Vertex<K, Tuple3<VV, Long, Long>>> vertex,
+							Collector<Vertex<K, Tuple3<VV, Long, Long>>> out) throws Exception {
+
+			final Iterator<Vertex<K, Tuple3<VV, Long, Long>>> vertexIter = vertex.iterator();
+		
+			if (vertexIter.hasNext()) {
+				Vertex<K, Tuple3<VV, Long, Long>> vertexWithDegrees = vertexIter.next();
+		
+				@SuppressWarnings("unchecked")
+				Iterator<Tuple2<?, Message>> downcastIter = (Iterator<Tuple2<?, Message>>) (Iterator<?>) messages.iterator();
+				messageIter.setSource(downcastIter);
+
+				vertexUpdateFunction.setInDegree(vertexWithDegrees.f1.f1);
+				vertexUpdateFunction.setOutDegree(vertexWithDegrees.f1.f2);
+
+				vertexUpdateFunction.setOutputWithDegrees(vertexWithDegrees, out);
+				vertexUpdateFunction.updateVertexFromVertexCentricIteration(vertexWithDegrees, messageIter);
+			}
+			else {
+				final Iterator<Tuple2<K, Message>> messageIter = messages.iterator();
+				if (messageIter.hasNext()) {
+					String message = "Target vertex does not exist!.";
+					try {
+						Tuple2<K, Message> next = messageIter.next();
+						message = "Target vertex '" + next.f0 + "' does not exist!.";
+					} catch (Throwable t) {}
+					throw new Exception(message);
+				} else {
+					throw new Exception();
+				}
+			}
+		}
+	}
+
+	/*
+	 * UDF that encapsulates the message sending function for graphs where the edges have an associated value.
+	 */
+	private static abstract class MessagingUdfWithEdgeValues<K, VVWithDegrees, VV, Message, EV>
+		extends RichCoGroupFunction<Edge<K, EV>, Vertex<K, VVWithDegrees>, Tuple2<K, Message>>
+		implements ResultTypeQueryable<Tuple2<K, Message>>
+	{
+		private static final long serialVersionUID = 1L;
+		
+		final MessagingFunction<K, VV, Message, EV> messagingFunction;
+		
+		private transient TypeInformation<Tuple2<K, Message>> resultType;
+	
+	
+		private MessagingUdfWithEdgeValues(MessagingFunction<K, VV, Message, EV> messagingFunction,
+				TypeInformation<Tuple2<K, Message>> resultType)
+		{
+			this.messagingFunction = messagingFunction;
+			this.resultType = resultType;
+		}
+		
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
+				this.messagingFunction.init(getIterationRuntimeContext());
+			}
+			
+			this.messagingFunction.preSuperstep();
+		}
+		
+		@Override
+		public void close() throws Exception {
+			this.messagingFunction.postSuperstep();
+		}
+		
+		@Override
+		public TypeInformation<Tuple2<K, Message>> getProducedType() {
+			return this.resultType;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class MessagingUdfWithEVsSimpleVV<K, VV, Message, EV>
+		extends MessagingUdfWithEdgeValues<K, VV, VV, Message, EV> {
+
+		private MessagingUdfWithEVsSimpleVV(MessagingFunction<K, VV, Message, EV> messagingFunction,
+			TypeInformation<Tuple2<K, Message>> resultType) {
+			super(messagingFunction, resultType);
+		}
+
+		@Override
+		public void coGroup(Iterable<Edge<K, EV>> edges,
+							Iterable<Vertex<K, VV>> state,
+							Collector<Tuple2<K, Message>> out) throws Exception {
+			final Iterator<Vertex<K, VV>> stateIter = state.iterator();
+		
+			if (stateIter.hasNext()) {
+				Vertex<K, VV> newVertexState = stateIter.next();
+				messagingFunction.set((Iterator<?>) edges.iterator(), out);
+				messagingFunction.sendMessages(newVertexState);
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class MessagingUdfWithEVsVVWithDegrees<K, VV, Message, EV>
+		extends MessagingUdfWithEdgeValues<K, Tuple3<VV, Long, Long>, VV, Message, EV> {
+
+		private Vertex<K, VV> nextVertex = new Vertex<K, VV>();
+
+		private MessagingUdfWithEVsVVWithDegrees(MessagingFunction<K, VV, Message, EV> messagingFunction,
+				TypeInformation<Tuple2<K, Message>> resultType) {
+			super(messagingFunction, resultType);
+		}
+
+		@Override
+		public void coGroup(Iterable<Edge<K, EV>> edges, Iterable<Vertex<K, Tuple3<VV, Long, Long>>> state,
+				Collector<Tuple2<K, Message>> out) throws Exception {
+
+			final Iterator<Vertex<K, Tuple3<VV, Long, Long>>> stateIter = state.iterator();
+		
+			if (stateIter.hasNext()) {
+				Vertex<K, Tuple3<VV, Long, Long>> vertexWithDegrees = stateIter.next();
+
+				nextVertex.setField(vertexWithDegrees.f0, 0);
+				nextVertex.setField(vertexWithDegrees.f1.f0, 1);
+
+				messagingFunction.setInDegree(vertexWithDegrees.f1.f1);
+				messagingFunction.setOutDegree(vertexWithDegrees.f1.f2);
+
+				messagingFunction.set((Iterator<?>) edges.iterator(), out);
+				messagingFunction.sendMessages(nextVertex);
+			}
+		}
+	}
+
+
+	// --------------------------------------------------------------------------------------------
+	//  UTIL methods
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Method that builds the messaging function using a coGroup operator for a simple vertex(without
+	 * degrees).
+	 * It afterwards configures the function with a custom name and broadcast variables.
+	 *
+	 * @param iteration
+	 * @param messageTypeInfo
+	 * @param whereArg the argument for the where within the coGroup
+	 * @param equalToArg the argument for the equalTo within the coGroup
+	 * @return the messaging function
+	 */
+	private CoGroupOperator<?, ?, Tuple2<K, Message>> buildMessagingFunction(
+			DeltaIteration<Vertex<K, VV>, Vertex<K, VV>> iteration,
+			TypeInformation<Tuple2<K, Message>> messageTypeInfo, int whereArg, int equalToArg) {
+
+		// build the messaging function (co group)
+		CoGroupOperator<?, ?, Tuple2<K, Message>> messages;
+		MessagingUdfWithEdgeValues<K, VV, VV, Message, EV> messenger =
+				new MessagingUdfWithEVsSimpleVV<K, VV, Message, EV>(messagingFunction, messageTypeInfo);
+
+		messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(whereArg)
+				.equalTo(equalToArg).with(messenger);
+
+		// configure coGroup message function with name and broadcast variables
+		messages = messages.name("Messaging");
+		if(this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getMessagingBcastVars()) {
+				messages = messages.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+
+		return messages;
+	}
+
+	/**
+	 * Method that builds the messaging function using a coGroup operator for a vertex
+	 * containing degree information.
+	 * It afterwards configures the function with a custom name and broadcast variables.
+	 *
+	 * @param iteration
+	 * @param messageTypeInfo
+	 * @param whereArg the argument for the where within the coGroup
+	 * @param equalToArg the argument for the equalTo within the coGroup
+	 * @return the messaging function
+	 */
+	private CoGroupOperator<?, ?, Tuple2<K, Message>> buildMessagingFunctionVerticesWithDegrees(
+			DeltaIteration<Vertex<K, Tuple3<VV, Long, Long>>, Vertex<K, Tuple3<VV, Long, Long>>> iteration,
+			TypeInformation<Tuple2<K, Message>> messageTypeInfo, int whereArg, int equalToArg) {
+
+		// build the messaging function (co group)
+		CoGroupOperator<?, ?, Tuple2<K, Message>> messages;
+		MessagingUdfWithEdgeValues<K, Tuple3<VV, Long, Long>, VV, Message, EV> messenger =
+				new MessagingUdfWithEVsVVWithDegrees<K, VV, Message, EV>(messagingFunction, messageTypeInfo);
+
+		messages = this.edgesWithValue.coGroup(iteration.getWorkset()).where(whereArg)
+				.equalTo(equalToArg).with(messenger);
+
+		// configure coGroup message function with name and broadcast variables
+		messages = messages.name("Messaging");
+
+		if (this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getMessagingBcastVars()) {
+				messages = messages.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+
+		return messages;
+	}
+
+	/**
+	 * Helper method which sets up an iteration with the given vertex value(either simple or with degrees)
+	 *
+	 * @param iteration
+	 */
+
+	private void setUpIteration(DeltaIteration<?, ?> iteration) {
+
+		// set up the iteration operator
+		if (this.configuration != null) {
+
+			iteration.name(this.configuration.getName("Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")"));
+			iteration.parallelism(this.configuration.getParallelism());
+			iteration.setSolutionSetUnManaged(this.configuration.isSolutionSetUnmanagedMemory());
+
+			// register all aggregators
+			for (Map.Entry<String, Aggregator<?>> entry : this.configuration.getAggregators().entrySet()) {
+				iteration.registerAggregator(entry.getKey(), entry.getValue());
+			}
+		}
+		else {
+			// no configuration provided; set default name
+			iteration.name("Vertex-centric iteration (" + updateFunction + " | " + messagingFunction + ")");
+		}
+	}
+
+	/**
+	 * Creates the operator that represents this vertex centric graph computation for a simple vertex.
+	 *
+	 * @param messagingDirection
+	 * @param messageTypeInfo
+	 * @return the operator
+	 */
+	private DataSet<Vertex<K, VV>> createResultSimpleVertex(EdgeDirection messagingDirection,
+		TypeInformation<Tuple2<K, Message>> messageTypeInfo) {
+
+		DataSet<Tuple2<K, Message>> messages;
+
+		TypeInformation<Vertex<K, VV>> vertexTypes = initialVertices.getType();
+
+		final DeltaIteration<Vertex<K, VV>,	Vertex<K, VV>> iteration =
+				initialVertices.iterateDelta(initialVertices, this.maximumNumberOfIterations, 0);
+				setUpIteration(iteration);
+
+		switch (messagingDirection) {
+			case IN:
+				messages = buildMessagingFunction(iteration, messageTypeInfo, 1, 0);
+				break;
+			case OUT:
+				messages = buildMessagingFunction(iteration, messageTypeInfo, 0, 0);
+				break;
+			case ALL:
+				messages = buildMessagingFunction(iteration, messageTypeInfo, 1, 0)
+						.union(buildMessagingFunction(iteration, messageTypeInfo, 0, 0)) ;
+				break;
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+
+		VertexUpdateUdf<K, VV, Message> updateUdf =
+				new VertexUpdateUdfSimpleVV<K, VV, Message>(updateFunction, vertexTypes);
+
+		// build the update function (co group)
+		CoGroupOperator<?, ?, Vertex<K, VV>> updates =
+				messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf);
+
+		configureUpdateFunction(updates);
+
+		return iteration.closeWith(updates, updates);
+	}
+
+	/**
+	 * Creates the operator that represents this vertex centric graph computation for a vertex with in
+	 * and out degrees added to the vertex value.
+	 *
+	 * @param graph
+	 * @param messagingDirection
+	 * @param messageTypeInfo
+	 * @return the operator
+	 */
+	@SuppressWarnings("serial")
+	private DataSet<Vertex<K, VV>> createResultVerticesWithDegrees(Graph<K, VV, EV> graph, EdgeDirection messagingDirection,
+			TypeInformation<Tuple2<K, Message>> messageTypeInfo) {
+
+		DataSet<Tuple2<K, Message>> messages;
+
+		this.updateFunction.setOptDegrees(this.configuration.isOptDegrees());
+
+		DataSet<Tuple2<K, Long>> inDegrees = graph.inDegrees();
+		DataSet<Tuple2<K, Long>> outDegrees = graph.outDegrees();
+
+		DataSet<Tuple3<K, Long, Long>> degrees = inDegrees.join(outDegrees).where(0).equalTo(0)
+				.with(new FlatJoinFunction<Tuple2<K, Long>, Tuple2<K, Long>, Tuple3<K, Long, Long>>() {
+
+					@Override
+					public void join(Tuple2<K, Long> first, Tuple2<K, Long> second,	Collector<Tuple3<K, Long, Long>> out) {
+						out.collect(new Tuple3<K, Long, Long>(first.f0, first.f1, second.f1));
+					}
+				}).withForwardedFieldsFirst("f0;f1").withForwardedFieldsSecond("f1");
+
+		DataSet<Vertex<K, Tuple3<VV, Long, Long>>> verticesWithDegrees = initialVertices
+				.join(degrees).where(0).equalTo(0)
+				.with(new FlatJoinFunction<Vertex<K,VV>, Tuple3<K,Long,Long>, Vertex<K, Tuple3<VV, Long, Long>>>() {
+					@Override
+					public void join(Vertex<K, VV> vertex, Tuple3<K, Long, Long> degrees,
+									Collector<Vertex<K, Tuple3<VV, Long, Long>>> out) throws Exception {
+
+						out.collect(new Vertex<K, Tuple3<VV, Long, Long>>(vertex.getId(),
+								new Tuple3<VV, Long, Long>(vertex.getValue(), degrees.f1, degrees.f2)));
+					}
+				}).withForwardedFieldsFirst("f0");
+
+		// add type info
+		TypeInformation<Vertex<K, Tuple3<VV, Long, Long>>> vertexTypes = verticesWithDegrees.getType();
+
+		final DeltaIteration<Vertex<K, Tuple3<VV, Long, Long>>,	Vertex<K, Tuple3<VV, Long, Long>>> iteration =
+				verticesWithDegrees.iterateDelta(verticesWithDegrees, this.maximumNumberOfIterations, 0);
+				setUpIteration(iteration);
+
+		switch (messagingDirection) {
+			case IN:
+				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 1, 0);
+				break;
+			case OUT:
+				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 0, 0);
+				break;
+			case ALL:
+				messages = buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 1, 0)
+						.union(buildMessagingFunctionVerticesWithDegrees(iteration, messageTypeInfo, 0, 0)) ;
+				break;
+			default:
+				throw new IllegalArgumentException("Illegal edge direction");
+		}
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		VertexUpdateUdf<K, Tuple3<VV, Long, Long>, Message> updateUdf =
+				new VertexUpdateUdfVVWithDegrees(updateFunction, vertexTypes);
+
+		// build the update function (co group)
+		CoGroupOperator<?, ?, Vertex<K, Tuple3<VV, Long, Long>>> updates =
+				messages.coGroup(iteration.getSolutionSet()).where(0).equalTo(0).with(updateUdf);
+
+		configureUpdateFunction(updates);
+
+		return iteration.closeWith(updates, updates).map(
+				new MapFunction<Vertex<K, Tuple3<VV, Long, Long>>, Vertex<K, VV>>() {
+
+					public Vertex<K, VV> map(Vertex<K, Tuple3<VV, Long, Long>> vertex) {
+						return new Vertex<K, VV>(vertex.getId(), vertex.getValue().f0);
+					}
+				});
+	}
+
+	private <VVWithDegree> void configureUpdateFunction(CoGroupOperator<?, ?, Vertex<K, VVWithDegree>> updates) {
+
+		// configure coGroup update function with name and broadcast variables
+		updates = updates.name("Vertex State Updates");
+		if (this.configuration != null) {
+			for (Tuple2<String, DataSet<?>> e : this.configuration.getUpdateBcastVars()) {
+				updates = updates.withBroadcastSet(e.f1, e.f0);
+			}
+		}
+
+		// let the operator know that we preserve the key field
+		updates.withForwardedFieldsFirst("0").withForwardedFieldsSecond("0");
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
new file mode 100644
index 0000000..248925b
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
@@ -0,0 +1,253 @@
+/*
+ * 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.spargel;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.functions.IterationRuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
+
+/**
+ * This class must be extended by functions that compute the state of the vertex depending on the old state and the
+ * incoming messages. The central method is {@link #updateVertex(Comparable, Object, MessageIterator)}, which is
+ * invoked once per vertex per superstep.
+ * 
+ * <K> The vertex key type.
+ * <VV> The vertex value type.
+ * <Message> The message type.
+ */
+public abstract class VertexUpdateFunction<K, VV, Message> implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// --------------------------------------------------------------------------------------------
+	//  Attributes that allow vertices to access their in/out degrees and the total number of vertices
+	//  inside an iteration.
+	// --------------------------------------------------------------------------------------------
+
+	private long numberOfVertices = -1L;
+
+	/**
+	 * Retrieves the number of vertices in the graph.
+	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
+	 * option has been set; -1 otherwise.
+	 */
+	public long getNumberOfVertices() {
+		return numberOfVertices;
+	}
+
+	void setNumberOfVertices(long numberOfVertices) {
+		this.numberOfVertices = numberOfVertices;
+	}
+
+	//---------------------------------------------------------------------------------------------
+
+	private boolean optDegrees;
+
+	boolean isOptDegrees() {
+		return optDegrees;
+	}
+
+	void setOptDegrees(boolean optDegrees) {
+		this.optDegrees = optDegrees;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Public API Methods
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * This method is invoked once per vertex per superstep. It receives the current state of the vertex, as well as
+	 * the incoming messages. It may set a new vertex state via {@link #setNewVertexValue(Object)}. If the vertex
+	 * state is changed, it will trigger the sending of messages via the {@link MessagingFunction}.
+	 * 
+	 * @param vertex The vertex.
+	 * @param inMessages The incoming messages to this vertex.
+	 * 
+	 * @throws Exception The computation may throw exceptions, which causes the superstep to fail.
+	 */
+	public abstract void updateVertex(Vertex<K, VV> vertex, MessageIterator<Message> inMessages) throws Exception;
+	
+	/**
+	 * This method is executed one per superstep before the vertex update function is invoked for each vertex.
+	 * 
+	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
+	 */
+	public void preSuperstep() throws Exception {}
+	
+	/**
+	 * This method is executed one per superstep after the vertex update function has been invoked for each vertex.
+	 * 
+	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
+	 */
+	public void postSuperstep() throws Exception {}
+	
+	/**
+	 * Sets the new value of this vertex. Setting a new value triggers the sending of outgoing messages from this vertex.
+	 *
+	 * This should be called at most once per updateVertex.
+	 * 
+	 * @param newValue The new vertex value.
+	 */
+	public void setNewVertexValue(VV newValue) {
+		if(setNewVertexValueCalled) {
+			throw new IllegalStateException("setNewVertexValue should only be called at most once per updateVertex");
+		}
+		setNewVertexValueCalled = true;
+		if(isOptDegrees()) {
+			outValWithDegrees.f1.f0 = newValue;
+			outWithDegrees.collect(outValWithDegrees);
+		} else {
+			outVal.setValue(newValue);
+			out.collect(outVal);
+		}
+	}
+	
+	/**
+	 * Gets the number of the superstep, starting at <tt>1</tt>.
+	 * 
+	 * @return The number of the current superstep.
+	 */
+	public int getSuperstepNumber() {
+		return this.runtimeContext.getSuperstepNumber();
+	}
+	
+	/**
+	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
+	 * all aggregates globally once per superstep and makes them available in the next superstep.
+	 * 
+	 * @param name The name of the aggregator.
+	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
+	 */
+	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
+		return this.runtimeContext.<T>getIterationAggregator(name);
+	}
+	
+	/**
+	 * Get the aggregated value that an aggregator computed in the previous iteration.
+	 * 
+	 * @param name The name of the aggregator.
+	 * @return The aggregated value of the previous iteration.
+	 */
+	public <T extends Value> T getPreviousIterationAggregate(String name) {
+		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
+	}
+	
+	/**
+	 * Gets the broadcast data set registered under the given name. Broadcast data sets
+	 * are available on all parallel instances of a function. They can be registered via
+	 * {@link org.apache.flink.graph.spargel.VertexCentricConfiguration#addBroadcastSetForUpdateFunction(String, org.apache.flink.api.java.DataSet)}.
+	 * 
+	 * @param name The name under which the broadcast set is registered.
+	 * @return The broadcast data set.
+	 */
+	public <T> Collection<T> getBroadcastSet(String name) {
+		return this.runtimeContext.<T>getBroadcastVariable(name);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  internal methods
+	// --------------------------------------------------------------------------------------------
+	
+	private IterationRuntimeContext runtimeContext;
+
+	private Collector<Vertex<K, VV>> out;
+	
+	private Collector<Vertex<K, Tuple3<VV, Long, Long>>> outWithDegrees;
+
+	private Vertex<K, VV> outVal;
+
+	private Vertex<K, Tuple3<VV, Long, Long>> outValWithDegrees;
+
+	private long inDegree = -1;
+
+	private long outDegree = -1;
+
+	private boolean setNewVertexValueCalled;
+
+	void init(IterationRuntimeContext context) {
+		this.runtimeContext = context;
+	}
+
+	void setOutput(Vertex<K, VV> outVal, Collector<Vertex<K, VV>> out) {
+		this.outVal = outVal;
+		this.out = out;
+		setNewVertexValueCalled = false;
+	}
+
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	<ValueWithDegree> void setOutputWithDegrees(Vertex<K, ValueWithDegree> outVal,
+			Collector out) {
+		this.outValWithDegrees = (Vertex<K, Tuple3<VV, Long, Long>>) outVal;
+		this.outWithDegrees = out;
+		setNewVertexValueCalled = false;
+	}
+
+	/**
+	 * Retrieves the vertex in-degree (number of in-coming edges).
+	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
+	 * option has been set; -1 otherwise. 
+	 */
+	public long getInDegree() {
+		return inDegree;
+	}
+
+	void setInDegree(long inDegree) {
+		this.inDegree = inDegree;
+	}
+
+	/**
+	 * Retrieve the vertex out-degree (number of out-going edges).
+	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
+	 * option has been set; -1 otherwise. 
+	 */
+	public long getOutDegree() {
+		return outDegree;
+	}
+
+	void setOutDegree(long outDegree) {
+		this.outDegree = outDegree;
+	}
+
+	/**
+	 * In order to hide the Tuple3(actualValue, inDegree, OutDegree) vertex value from the user,
+	 * another function will be called from {@link org.apache.flink.graph.spargel.VertexCentricIteration}.
+	 *
+	 * This function will retrieve the vertex from the vertexState and will set its degrees, afterwards calling
+	 * the regular updateVertex function.
+	 *
+	 * @param vertexState
+	 * @param inMessages
+	 * @throws Exception
+	 */
+	@SuppressWarnings("unchecked")
+	<VertexWithDegree> void updateVertexFromVertexCentricIteration(Vertex<K, VertexWithDegree> vertexState,
+												MessageIterator<Message> inMessages) throws Exception {
+
+		Vertex<K, VV> vertex = new Vertex<K, VV>(vertexState.f0,
+				((Tuple3<VV, Long, Long>)vertexState.getValue()).f0);
+
+		updateVertex(vertex, inMessages);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
new file mode 100644
index 0000000..0e085b4
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/EdgeToTuple3Map.java
@@ -0,0 +1,35 @@
+/*
+ * 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.api.java.functions.FunctionAnnotation.ForwardedFields;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+
+@ForwardedFields("f0; f1; f2")
+public class EdgeToTuple3Map<K, EV> implements MapFunction<Edge<K, EV>, Tuple3<K, K, EV>> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Tuple3<K, K, EV> map(Edge<K, EV> edge) {
+		return new Tuple3<K, K, EV>(edge.f0, edge.f1, edge.f2);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/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
new file mode 100644
index 0000000..2bd4719
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/NullValueEdgeMapper.java
@@ -0,0 +1,32 @@
+/*
+ * 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();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
new file mode 100644
index 0000000..e51362b
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple2ToVertexMap.java
@@ -0,0 +1,35 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+
+@ForwardedFields("f0; f1")
+public class Tuple2ToVertexMap<K, VV> implements MapFunction<Tuple2<K, VV>, Vertex<K, VV>> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Vertex<K, VV> map(Tuple2<K, VV> vertex) {
+		return new Vertex<K, VV>(vertex.f0, vertex.f1);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.java
new file mode 100644
index 0000000..0db9a51
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Tuple3ToEdgeMap.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.utils;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+
+/**
+ * create an Edge DataSetfrom a Tuple3 dataset
+ *
+ * @param <K>
+ * @param <EV>
+ */
+@ForwardedFields("f0; f1; f2")
+public class Tuple3ToEdgeMap<K, EV> implements MapFunction<Tuple3<K, K, EV>, Edge<K, EV>> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Edge<K, EV> map(Tuple3<K, K, EV> tuple) {
+		return new Edge<K, EV>(tuple.f0, tuple.f1, tuple.f2);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
new file mode 100644
index 0000000..04d1f47
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/VertexToTuple2Map.java
@@ -0,0 +1,35 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
+
+@ForwardedFields("f0; f1")
+public class VertexToTuple2Map<K, VV> implements MapFunction<Vertex<K, VV>, Tuple2<K, VV>> {
+
+	private static final long serialVersionUID = 1L;
+
+	public Tuple2<K, VV> map(Vertex<K, VV> vertex) {
+		return new Tuple2<K, VV>(vertex.f0, vertex.f1);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
new file mode 100644
index 0000000..75b672c
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/GraphValidator.java
@@ -0,0 +1,37 @@
+/*
+ * 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.validation;
+
+import java.io.Serializable;
+
+import org.apache.flink.graph.Graph;
+
+/**
+ * A utility for defining validation criteria for different types of Graphs.
+ * 
+ * @param <K> the vertex key type
+ * @param <VV> the vertex value type
+ * @param <EV> the edge value type
+ */
+@SuppressWarnings("serial")
+public abstract class GraphValidator<K, VV, EV>	implements Serializable {
+
+	public abstract boolean validate(Graph<K, VV, EV> graph) throws Exception;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.java
new file mode 100644
index 0000000..33d469b
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/validation/InvalidVertexIdsValidator.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.validation;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.util.Collector;
+
+@SuppressWarnings("serial")
+public class InvalidVertexIdsValidator<K, VV, EV> extends GraphValidator<K, VV, EV> {
+
+	/**
+	 * Checks that the edge set input contains valid vertex Ids, i.e. that they
+	 * also exist in the vertex input set.
+	 * 
+	 * @return a boolean stating whether a graph is valid
+	 *         with respect to its vertex ids.
+	 */
+	@Override
+	public boolean validate(Graph<K, VV, EV> graph) throws Exception {
+		DataSet<Tuple1<K>> edgeIds = graph.getEdges()
+				.flatMap(new MapEdgeIds<K, EV>()).distinct();
+		DataSet<K> invalidIds = graph.getVertices().coGroup(edgeIds).where(0)
+				.equalTo(0).with(new GroupInvalidIds<K, VV>()).first(1);
+
+		return invalidIds.map(new KToTupleMap<K>()).count() == 0;
+	}
+
+	private static final class MapEdgeIds<K, EV> implements FlatMapFunction<Edge<K, EV>, Tuple1<K>> {
+		public void flatMap(Edge<K, EV> edge, Collector<Tuple1<K>> out) {
+			out.collect(new Tuple1<K>(edge.f0));
+			out.collect(new Tuple1<K>(edge.f1));
+		}
+	}
+
+	private static final class GroupInvalidIds<K, VV> implements CoGroupFunction<Vertex<K, VV>, Tuple1<K>, K> {
+		public void coGroup(Iterable<Vertex<K, VV>> vertexId,
+				Iterable<Tuple1<K>> edgeId, Collector<K> out) {
+			if (!(vertexId.iterator().hasNext())) {
+				// found an id that doesn't exist in the vertex set
+				out.collect(edgeId.iterator().next().f0);
+			}
+		}
+	}
+
+	private static final class KToTupleMap<K> implements MapFunction<K, Tuple1<K>> {
+		public Tuple1<K> map(K key) throws Exception {
+			return new Tuple1<K>(key);
+		}
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
new file mode 100644
index 0000000..2ad203f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSACompilerTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.gsa;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
+import org.apache.flink.optimizer.dataproperties.PartitioningProperty;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.PlanNode;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+
+public class GSACompilerTest extends CompilerTestBase {
+
+	private static final long serialVersionUID = 1L;
+
+	@Test
+	public void testGSACompiler() {
+		try {
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			env.setParallelism(DEFAULT_PARALLELISM);
+			// compose test program
+			{
+				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<Long, Long, NullValue>(
+						1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
+
+				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
+
+				DataSet<Vertex<Long, Long>> result = graph.runGatherSumApplyIteration(
+						new GatherNeighborIds(), new SelectMinId(),
+						new UpdateComponentId(), 100).getVertices();
+				
+				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			}
+			
+			Plan p = env.createProgramPlan("GSA Connected Components");
+			OptimizedPlan op = compileNoStats(p);
+			
+			// check the sink
+			SinkPlanNode sink = op.getDataSinks().iterator().next();
+			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
+			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
+			assertEquals(PartitioningProperty.HASH_PARTITIONED, sink.getGlobalProperties().getPartitioning());
+			
+			// check the iteration
+			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
+			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
+			
+			// check the solution set join and the delta
+			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
+			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update function preserves the partitioning
+			
+			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
+			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
+			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
+			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
+			
+			// check the workset set join
+			SingleInputPlanNode sumReducer = (SingleInputPlanNode) ssJoin.getInput1().getSource();
+			SingleInputPlanNode gatherMapper = (SingleInputPlanNode) sumReducer.getInput().getSource();
+			DualInputPlanNode edgeJoin = (DualInputPlanNode) gatherMapper.getInput().getSource(); 
+			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
+			// input1 is the workset
+			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput1().getShipStrategy());
+			// input2 is the edges
+			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput2().getShipStrategy());
+			assertTrue(edgeJoin.getInput2().getTempMode().isCached());
+
+			assertEquals(new FieldList(0), edgeJoin.getInput2().getShipStrategyKeys());
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitVertices	implements MapFunction<Long, Long> {
+
+		public Long map(Long vertexId) {
+			return vertexId;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
+
+		public Long gather(Neighbor<Long, NullValue> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
+
+		public Long sum(Long newValue, Long currentValue) {
+			return Math.min(newValue, currentValue);
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class UpdateComponentId extends ApplyFunction<Long, Long, Long> {
+
+		public void apply(Long summedValue, Long origValue) {
+			if (summedValue < origValue) {
+				setResult(summedValue);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
new file mode 100644
index 0000000..ced7508
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/gsa/GSATranslationTest.java
@@ -0,0 +1,164 @@
+/*
+ * 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.gsa;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+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.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.operators.DeltaIterationResultSet;
+import org.apache.flink.api.java.operators.SingleInputUdfOperator;
+import org.apache.flink.api.java.operators.TwoInputUdfOperator;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+
+public class GSATranslationTest {
+
+	@Test
+	public void testTranslation() {
+		try {
+			final String ITERATION_NAME = "Test Name";
+			
+			final String AGGREGATOR_NAME = "AggregatorName";
+			
+			final String BC_SET_GATHER_NAME = "gather messages";
+			
+			final String BC_SET_SUM_NAME = "sum updates";
+
+			final String BC_SET_APLLY_NAME = "apply updates";
+
+			final int NUM_ITERATIONS = 13;
+			
+			final int ITERATION_parallelism = 77;
+			
+			
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			
+			DataSet<Long> bcGather = env.fromElements(1L);
+			DataSet<Long> bcSum = env.fromElements(1L);
+			DataSet<Long> bcApply = env.fromElements(1L);
+
+			DataSet<Vertex<Long, Long>> result;
+
+			// ------------ construct the test program ------------------
+			{
+
+				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple3<Long, Long, NullValue>(
+						1L, 2L, NullValue.getInstance())).map(new Tuple3ToEdgeMap<Long, NullValue>());
+
+				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, new InitVertices(), env);
+
+				GSAConfiguration parameters = new GSAConfiguration();
+
+				parameters.registerAggregator(AGGREGATOR_NAME, new LongSumAggregator());
+				parameters.setName(ITERATION_NAME);
+				parameters.setParallelism(ITERATION_parallelism);
+				parameters.addBroadcastSetForGatherFunction(BC_SET_GATHER_NAME, bcGather);
+				parameters.addBroadcastSetForSumFunction(BC_SET_SUM_NAME, bcSum);
+				parameters.addBroadcastSetForApplyFunction(BC_SET_APLLY_NAME, bcApply);
+
+				result = graph.runGatherSumApplyIteration(
+						new GatherNeighborIds(), new SelectMinId(),
+						new UpdateComponentId(), NUM_ITERATIONS, parameters).getVertices();
+				
+				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			}
+			
+			
+			// ------------- validate the java program ----------------
+			
+			assertTrue(result instanceof DeltaIterationResultSet);
+			
+			DeltaIterationResultSet<?, ?> resultSet = (DeltaIterationResultSet<?, ?>) result;
+			DeltaIteration<?, ?> iteration = (DeltaIteration<?, ?>) resultSet.getIterationHead();
+			
+			// check the basic iteration properties
+			assertEquals(NUM_ITERATIONS, resultSet.getMaxIterations());
+			assertArrayEquals(new int[] {0}, resultSet.getKeyPositions());
+			assertEquals(ITERATION_parallelism, iteration.getParallelism());
+			assertEquals(ITERATION_NAME, iteration.getName());
+			
+			assertEquals(AGGREGATOR_NAME, iteration.getAggregators().getAllRegisteredAggregators().iterator().next().getName());
+			
+			// validate that the semantic properties are set as they should
+			TwoInputUdfOperator<?, ?, ?, ?> solutionSetJoin = (TwoInputUdfOperator<?, ?, ?, ?>) resultSet.getNextWorkset();
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(0, 0).contains(0));
+			assertTrue(solutionSetJoin.getSemanticProperties().getForwardingTargetFields(1, 0).contains(0));
+
+			SingleInputUdfOperator<?, ?, ?> sumReduce = (SingleInputUdfOperator<?, ?, ?>) solutionSetJoin.getInput1();
+			SingleInputUdfOperator<?, ?, ?> gatherMap = (SingleInputUdfOperator<?, ?, ?>) sumReduce.getInput();
+
+			// validate that the broadcast sets are forwarded
+			assertEquals(bcGather, gatherMap.getBroadcastSets().get(BC_SET_GATHER_NAME));
+			assertEquals(bcSum, sumReduce.getBroadcastSets().get(BC_SET_SUM_NAME));
+			assertEquals(bcApply, solutionSetJoin.getBroadcastSets().get(BC_SET_APLLY_NAME));
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitVertices	implements MapFunction<Long, Long> {
+
+		public Long map(Long vertexId) {
+			return vertexId;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
+
+		public Long gather(Neighbor<Long, NullValue> neighbor) {
+			return neighbor.getNeighborValue();
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
+
+		public Long sum(Long newValue, Long currentValue) {
+			return Math.min(newValue, currentValue);
+		}
+	};
+
+	@SuppressWarnings("serial")
+	private static final class UpdateComponentId extends ApplyFunction<Long, Long, Long> {
+
+		public void apply(Long summedValue, Long origValue) {
+			if (summedValue < origValue) {
+				setResult(summedValue);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
new file mode 100644
index 0000000..7a8143a
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/spargel/SpargelCompilerTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.spargel;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.junit.Test;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.PlanNode;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.runtime.operators.util.LocalStrategy;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.graph.library.ConnectedComponents;
+import org.apache.flink.graph.utils.Tuple2ToVertexMap;
+
+
+public class SpargelCompilerTest extends CompilerTestBase {
+
+	private static final long serialVersionUID = 1L;
+
+	@SuppressWarnings("serial")
+	@Test
+	public void testSpargelCompiler() {
+		try {
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			env.setParallelism(DEFAULT_PARALLELISM);
+			// compose test program
+			{
+
+				DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
+						new Tuple2<Long, Long>(1L, 1L), new Tuple2<Long, Long>(2L, 2L))
+						.map(new Tuple2ToVertexMap<Long, Long>());
+
+				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<Long, Long>(1L, 2L))
+					.map(new MapFunction<Tuple2<Long,Long>, Edge<Long, NullValue>>() {
+
+						public Edge<Long, NullValue> map(Tuple2<Long, Long> edge) {
+							return new Edge<Long, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
+						}
+				});
+
+				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
+				
+				DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
+						new ConnectedComponents.CCUpdater<Long>(),
+						new ConnectedComponents.CCMessenger<Long>(), 100)
+						.getVertices();
+				
+				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+			}
+			
+			Plan p = env.createProgramPlan("Spargel Connected Components");
+			OptimizedPlan op = compileNoStats(p);
+			
+			// check the sink
+			SinkPlanNode sink = op.getDataSinks().iterator().next();
+			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
+			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
+			
+			// check the iteration
+			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
+			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
+			
+			// check the solution set join and the delta
+			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
+			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update functions preserves the partitioning
+			
+			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
+			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
+			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
+			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
+			
+			// check the workset set join
+			DualInputPlanNode edgeJoin = (DualInputPlanNode) ssJoin.getInput1().getSource();
+			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
+			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput1().getShipStrategy());
+			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput2().getShipStrategy());
+			assertTrue(edgeJoin.getInput1().getTempMode().isCached());
+			
+			assertEquals(new FieldList(0), edgeJoin.getInput1().getShipStrategyKeys());
+			
+			// check that the initial partitioning is pushed out of the loop
+			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput1().getShipStrategy());
+			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput2().getShipStrategy());
+			assertEquals(new FieldList(0), iteration.getInput1().getShipStrategyKeys());
+			assertEquals(new FieldList(0), iteration.getInput2().getShipStrategyKeys());
+			
+			// check that the initial workset sort is outside the loop
+			assertEquals(LocalStrategy.SORT, iteration.getInput2().getLocalStrategy());
+			assertEquals(new FieldList(0), iteration.getInput2().getLocalStrategyKeys());
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@SuppressWarnings("serial")
+	@Test
+	public void testSpargelCompilerWithBroadcastVariable() {
+		try {
+			final String BC_VAR_NAME = "borat variable";
+			
+			
+			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			env.setParallelism(DEFAULT_PARALLELISM);
+			// compose test program
+			{
+				DataSet<Long> bcVar = env.fromElements(1L);
+
+				DataSet<Vertex<Long, Long>> initialVertices = env.fromElements(
+						new Tuple2<Long, Long>(1L, 1L), new Tuple2<Long, Long>(2L, 2L))
+						.map(new Tuple2ToVertexMap<Long, Long>());
+
+				DataSet<Edge<Long, NullValue>> edges = env.fromElements(new Tuple2<Long, Long>(1L, 2L))
+						.map(new MapFunction<Tuple2<Long,Long>, Edge<Long, NullValue>>() {
+
+							public Edge<Long, NullValue> map(Tuple2<Long, Long> edge) {
+								return new Edge<Long, NullValue>(edge.f0, edge.f1, NullValue.getInstance());
+							}
+					});
+
+				Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
+
+				VertexCentricConfiguration parameters = new VertexCentricConfiguration();
+				parameters.addBroadcastSetForMessagingFunction(BC_VAR_NAME, bcVar);
+				parameters.addBroadcastSetForUpdateFunction(BC_VAR_NAME, bcVar);
+
+				DataSet<Vertex<Long, Long>> result = graph.runVertexCentricIteration(
+						new ConnectedComponents.CCUpdater<Long>(),
+						new ConnectedComponents.CCMessenger<Long>(), 100)
+						.getVertices();
+					
+				result.output(new DiscardingOutputFormat<Vertex<Long, Long>>());
+
+			}
+			
+			Plan p = env.createProgramPlan("Spargel Connected Components");
+			OptimizedPlan op = compileNoStats(p);
+			
+			// check the sink
+			SinkPlanNode sink = op.getDataSinks().iterator().next();
+			assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
+			assertEquals(DEFAULT_PARALLELISM, sink.getParallelism());
+			
+			// check the iteration
+			WorksetIterationPlanNode iteration = (WorksetIterationPlanNode) sink.getInput().getSource();
+			assertEquals(DEFAULT_PARALLELISM, iteration.getParallelism());
+			
+			// check the solution set join and the delta
+			PlanNode ssDelta = iteration.getSolutionSetDeltaPlanNode();
+			assertTrue(ssDelta instanceof DualInputPlanNode); // this is only true if the update functions preserves the partitioning
+			
+			DualInputPlanNode ssJoin = (DualInputPlanNode) ssDelta;
+			assertEquals(DEFAULT_PARALLELISM, ssJoin.getParallelism());
+			assertEquals(ShipStrategyType.PARTITION_HASH, ssJoin.getInput1().getShipStrategy());
+			assertEquals(new FieldList(0), ssJoin.getInput1().getShipStrategyKeys());
+			
+			// check the workset set join
+			DualInputPlanNode edgeJoin = (DualInputPlanNode) ssJoin.getInput1().getSource();
+			assertEquals(DEFAULT_PARALLELISM, edgeJoin.getParallelism());
+			assertEquals(ShipStrategyType.PARTITION_HASH, edgeJoin.getInput1().getShipStrategy());
+			assertEquals(ShipStrategyType.FORWARD, edgeJoin.getInput2().getShipStrategy());
+			assertTrue(edgeJoin.getInput1().getTempMode().isCached());
+			
+			assertEquals(new FieldList(0), edgeJoin.getInput1().getShipStrategyKeys());
+			
+			// check that the initial partitioning is pushed out of the loop
+			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput1().getShipStrategy());
+			assertEquals(ShipStrategyType.PARTITION_HASH, iteration.getInput2().getShipStrategy());
+			assertEquals(new FieldList(0), iteration.getInput1().getShipStrategyKeys());
+			assertEquals(new FieldList(0), iteration.getInput2().getShipStrategyKeys());
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[15/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
new file mode 100644
index 0000000..5aa9f26
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/MusicProfilesITCase.java
@@ -0,0 +1,101 @@
+/*
+ * 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.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+import org.apache.flink.graph.example.MusicProfiles;
+import org.apache.flink.graph.example.utils.MusicProfilesData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.After;
+import org.junit.Assert;
+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;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+@RunWith(Parameterized.class)
+public class MusicProfilesITCase extends MultipleProgramsTestBase {
+
+	private String tripletsPath;
+
+	private String mismatchesPath;
+
+	private String topSongsResultPath;
+
+	private String communitiesResultPath;
+
+	private String expectedTopSongs;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	public MusicProfilesITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Before
+	public void before() throws Exception {
+		topSongsResultPath = tempFolder.newFile().toURI().toString();
+		communitiesResultPath = tempFolder.newFile().toURI().toString();
+
+		File tripletsFile = tempFolder.newFile();
+		Files.write(MusicProfilesData.USER_SONG_TRIPLETS, tripletsFile, Charsets.UTF_8);
+		tripletsPath = tripletsFile.toURI().toString();
+
+		File mismatchesFile = tempFolder.newFile();
+		Files.write(MusicProfilesData.MISMATCHES, mismatchesFile, Charsets.UTF_8);
+		mismatchesPath = mismatchesFile.toURI().toString();
+	}
+
+	@Test
+	public void testMusicProfilesExample() throws Exception {
+		MusicProfiles.main(new String[]{tripletsPath, mismatchesPath, topSongsResultPath, "0", communitiesResultPath,
+				MusicProfilesData.MAX_ITERATIONS + ""});
+		expectedTopSongs = MusicProfilesData.TOP_SONGS_RESULT;
+	}
+
+	@After
+	public void after() throws Exception {
+		compareResultsByLinesInMemory(expectedTopSongs, topSongsResultPath);
+
+		ArrayList<String> list = new ArrayList<String>();
+		readAllResultLines(list, communitiesResultPath, new String[]{}, false);
+
+		String[] result = list.toArray(new String[list.size()]);
+		Arrays.sort(result);
+
+		// check that user_1 and user_2 are in the same community
+		Assert.assertEquals("users 1 and 2 are not in the same community",
+				result[0].substring(7), result[1].substring(7));
+
+		// check that user_3, user_4 and user_5 are in the same community
+		Assert.assertEquals("users 3 and 4 are not in the same community",
+				result[2].substring(7), result[3].substring(7));
+		Assert.assertEquals("users 4 and 5 are not in the same community",
+				result[3].substring(7), result[4].substring(7));
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.java
new file mode 100644
index 0000000..d8f8c8f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/example/SingleSourceShortestPathsITCase.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.test.example;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+import org.apache.flink.graph.example.GSASingleSourceShortestPaths;
+import org.apache.flink.graph.example.SingleSourceShortestPaths;
+import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+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 SingleSourceShortestPathsITCase extends MultipleProgramsTestBase {
+
+    private String edgesPath;
+
+    private String resultPath;
+
+    private String expected;
+
+    @Rule
+    public TemporaryFolder tempFolder = new TemporaryFolder();
+
+    public SingleSourceShortestPathsITCase(TestExecutionMode mode) {
+        super(mode);
+    }
+
+    @Before
+    public void before() throws Exception {
+        resultPath = tempFolder.newFile().toURI().toString();
+
+        File edgesFile = tempFolder.newFile();
+        Files.write(SingleSourceShortestPathsData.EDGES, edgesFile, Charsets.UTF_8);
+        edgesPath = edgesFile.toURI().toString();
+    }
+
+    @Test
+    public void testSSSPExample() throws Exception {
+        SingleSourceShortestPaths.main(new String[]{SingleSourceShortestPathsData.SRC_VERTEX_ID + "",
+                edgesPath, resultPath, 10 + ""});
+        expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS;
+    }
+
+    @Test
+    public void testGSASSSPExample() throws Exception {
+        GSASingleSourceShortestPaths.main(new String[]{SingleSourceShortestPathsData.SRC_VERTEX_ID + "",
+                edgesPath, resultPath, 10 + ""});
+        expected = SingleSourceShortestPathsData.RESULTED_SINGLE_SOURCE_SHORTEST_PATHS;
+    }
+
+    @After
+    public void after() throws Exception {
+        compareResultsByLinesInMemory(expected, resultPath);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
new file mode 100644
index 0000000..421eaa9
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/CommunityDetectionITCase.java
@@ -0,0 +1,82 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.CommunityDetectionData;
+import org.apache.flink.graph.library.CommunityDetection;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class CommunityDetectionITCase extends MultipleProgramsTestBase {
+
+	public CommunityDetectionITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private String expected;
+
+	@Test
+	public void testSingleIteration() throws Exception {
+		/*
+		 * Test one iteration of the Simple Community Detection Example
+		 */
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
+				CommunityDetectionData.getSimpleEdgeDataSet(env), new InitLabels(), env);
+
+        List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
+        		.getVertices().collect();
+
+		expected = CommunityDetectionData.COMMUNITIES_SINGLE_ITERATION;
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testTieBreaker() throws Exception {
+		/*
+		 * Test one iteration of the Simple Community Detection Example where a tie must be broken
+		 */
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Double> inputGraph = Graph.fromDataSet(
+				CommunityDetectionData.getTieEdgeDataSet(env), new InitLabels(), env);
+
+        List<Vertex<Long, Long>> result = inputGraph.run(new CommunityDetection<Long>(1, CommunityDetectionData.DELTA))
+        		.getVertices().collect();
+		expected = CommunityDetectionData.COMMUNITIES_WITH_TIE;
+		compareResultAsTuples(result, expected);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitLabels implements MapFunction<Long, Long>{
+
+		public Long map(Long id) {
+			return id;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
new file mode 100644
index 0000000..9eb7387
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/ConnectedComponentsWithRandomisedEdgesITCase.java
@@ -0,0 +1,94 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.library.ConnectedComponents;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.types.NullValue;
+
+import java.io.BufferedReader;
+
+@SuppressWarnings("serial")
+public class ConnectedComponentsWithRandomisedEdgesITCase extends JavaProgramTestBase {
+
+	private static final long SEED = 9487520347802987L;
+
+	private static final int NUM_VERTICES = 1000;
+
+	private static final int NUM_EDGES = 10000;
+
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempFilePath("results");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Long> vertexIds = env.generateSequence(1, NUM_VERTICES);
+		DataSet<String> edgeString = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED).split("\n"));
+
+		DataSet<Edge<Long, NullValue>> edges = edgeString.map(new EdgeParser());
+
+		DataSet<Vertex<Long, Long>> initialVertices = vertexIds.map(new IdAssigner());
+
+		Graph<Long, Long, NullValue> graph = Graph.fromDataSet(initialVertices, edges, env);
+
+		DataSet<Vertex<Long, Long>> result = graph.run(new ConnectedComponents<Long, NullValue>(100));
+
+		result.writeAsCsv(resultPath, "\n", " ");
+		env.execute();
+	}
+
+	/**
+	 * A map function that takes a Long value and creates a 2-tuple out of it:
+	 * <pre>(Long value) -> (value, value)</pre>
+	 */
+	public static final class IdAssigner implements MapFunction<Long, Vertex<Long, Long>> {
+		@Override
+		public Vertex<Long, Long> map(Long value) {
+			return new Vertex<Long, Long>(value, value);
+		}
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			ConnectedComponentsData.checkOddEvenResult(reader);
+		}
+	}
+
+	public static final class EdgeParser extends RichMapFunction<String, Edge<Long, NullValue>> {
+		public Edge<Long, NullValue> map(String value) {
+			String[] nums = value.split(" ");
+			return new Edge<Long, NullValue>(Long.parseLong(nums[0]), Long.parseLong(nums[1]),
+					NullValue.getInstance());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
new file mode 100644
index 0000000..8785b0d
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/LabelPropagationITCase.java
@@ -0,0 +1,78 @@
+/*
+ * 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.library;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.LabelPropagationData;
+import org.apache.flink.graph.library.LabelPropagation;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class LabelPropagationITCase extends MultipleProgramsTestBase {
+
+	public LabelPropagationITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testSingleIteration() throws Exception {
+		/*
+		 * Test one iteration of label propagation example with a simple graph
+		 */
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
+				LabelPropagationData.getDefaultVertexSet(env),
+				LabelPropagationData.getDefaultEdgeDataSet(env), env);
+
+        List<Vertex<Long, Long>> result = inputGraph.run(new LabelPropagation<Long, NullValue>(1))
+        		.collect();
+
+		expectedResult = LabelPropagationData.LABELS_AFTER_1_ITERATION;
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testTieBreaker() throws Exception {
+		/*
+		 * Test the label propagation example where a tie must be broken
+		 */
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, NullValue> inputGraph = Graph.fromDataSet(
+				LabelPropagationData.getTieVertexSet(env),
+				LabelPropagationData.getTieEdgeDataSet(env), env);
+
+        List<Vertex<Long, Long>> result = inputGraph.run(new LabelPropagation<Long, NullValue>(1))
+        		.collect();
+
+		expectedResult = LabelPropagationData.LABELS_WITH_TIE;
+		compareResultAsTuples(result, expectedResult);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
new file mode 100644
index 0000000..94c7713
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/PageRankITCase.java
@@ -0,0 +1,132 @@
+/*
+ * 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.library;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.example.utils.PageRankData;
+import org.apache.flink.graph.library.GSAPageRank;
+import org.apache.flink.graph.library.PageRank;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class PageRankITCase extends MultipleProgramsTestBase {
+
+	public PageRankITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	private String expectedResult;
+
+	@Test
+	public void testPageRankWithThreeIterations() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
+				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
+
+        List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 3))
+        		.collect();
+        
+        compareWithDelta(result, expectedResult, 0.01);
+	}
+
+	@Test
+	public void testGSAPageRankWithThreeIterations() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
+				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
+
+        List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 3))
+        		.collect();
+        
+        compareWithDelta(result, expectedResult, 0.01);
+	}
+
+	@Test
+	public void testPageRankWithThreeIterationsAndNumOfVertices() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
+				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
+
+        List<Vertex<Long, Double>> result = inputGraph.run(new PageRank<Long>(0.85, 5, 3))
+        		.collect();
+        
+        compareWithDelta(result, expectedResult, 0.01);
+	}
+
+	@Test
+	public void testGSAPageRankWithThreeIterationsAndNumOfVertices() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Double, Double> inputGraph = Graph.fromDataSet(
+				PageRankData.getDefaultEdgeDataSet(env), new InitMapper(), env);
+
+        List<Vertex<Long, Double>> result = inputGraph.run(new GSAPageRank<Long>(0.85, 5, 3))
+        		.collect();
+        
+        compareWithDelta(result, expectedResult, 0.01);
+	}
+
+	private void compareWithDelta(List<Vertex<Long, Double>> result,
+			String expectedResult, double delta) {
+
+		String resultString = "";
+        for (Vertex<Long, Double> v : result) {
+        	resultString += v.f0.toString() + "," + v.f1.toString() +"\n";
+        }
+        
+		expectedResult = PageRankData.RANKS_AFTER_3_ITERATIONS;
+		String[] expected = expectedResult.isEmpty() ? new String[0] : expectedResult.split("\n");
+
+		String[] resultArray = resultString.isEmpty() ? new String[0] : resultString.split("\n");
+
+		Arrays.sort(expected);
+        Arrays.sort(resultArray);
+
+		for (int i = 0; i < expected.length; i++) {
+			String[] expectedFields = expected[i].split(",");
+			String[] resultFields = resultArray[i].split(",");
+
+			double expectedPayLoad = Double.parseDouble(expectedFields[1]);
+			double resultPayLoad = Double.parseDouble(resultFields[1]);
+
+			Assert.assertTrue("Values differ by more than the permissible delta",
+					Math.abs(expectedPayLoad - resultPayLoad) < delta);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class InitMapper implements MapFunction<Long, Double> {
+		public Double map(Long value) {
+			return 1.0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.java
new file mode 100644
index 0000000..1d9ab9f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/library/TriangleCountITCase.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.test.library;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.example.utils.TriangleCountData;
+import org.apache.flink.graph.library.GSATriangleCount;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class TriangleCountITCase extends MultipleProgramsTestBase {
+
+	private String expectedResult;
+
+	public TriangleCountITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testGSATriangleCount() throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(TriangleCountData.getDefaultEdgeDataSet(env),
+				env).getUndirected();
+
+		List<Integer> numberOfTriangles = graph.run(new GSATriangleCount<Long, NullValue, NullValue>()).collect();
+		expectedResult = TriangleCountData.RESULTED_NUMBER_OF_TRIANGLES;
+
+		Assert.assertEquals(numberOfTriangles.get(0).intValue(), Integer.parseInt(expectedResult));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
new file mode 100644
index 0000000..b2744f9
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesITCase.java
@@ -0,0 +1,178 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+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.Graph;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class DegreesITCase extends MultipleProgramsTestBase {
+
+	public DegreesITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+
+	@Test
+	public void testOutDegrees() throws Exception {
+		/*
+		* Test outDegrees()
+		*/
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+       
+        
+        expectedResult = "1,2\n" +
+                    "2,1\n" +
+                    "3,2\n" +
+                    "4,1\n" +
+                    "5,1\n";
+        
+        compareResultAsTuples(result, expectedResult);
+        
+    }
+
+	@Test
+	public void testOutDegreesWithNoOutEdges() throws Exception {
+		/*
+		 * Test outDegrees() no outgoing edges
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env);
+
+        
+        
+        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+        
+        expectedResult = "1,3\n" +
+                "2,1\n" +
+                "3,1\n" +
+                "4,1\n" +
+                "5,0\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testInDegrees() throws Exception {
+		/*
+		 * Test inDegrees()
+		 */
+	    final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+	    Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+	            TestGraphUtils.getLongLongEdgeData(env), env);
+
+
+        DataSet<Tuple2<Long,Long>> data =graph.inDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+	    
+	    expectedResult = "1,1\n" +
+		            "2,1\n" +
+		            "3,2\n" +
+		            "4,1\n" +
+		            "5,2\n";
+	    compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testInDegreesWithNoInEdge() throws Exception {
+		/*
+		 * Test inDegrees() no ingoing edge
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeDataWithZeroDegree(env), env);
+
+        DataSet<Tuple2<Long,Long>> data =graph.inDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+        
+        expectedResult = "1,0\n" +
+	                "2,1\n" +
+	                "3,1\n" +
+	                "4,1\n" +
+	                "5,3\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testGetDegrees() throws Exception {
+		/*
+		 * Test getDegrees()
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Tuple2<Long,Long>> data =graph.getDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+        
+        expectedResult = "1,3\n" +
+	                "2,2\n" +
+	                "3,4\n" +
+	                "4,2\n" +
+	                "5,3\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testGetDegreesWithDisconnectedData() throws Exception {
+        /*
+		 * Test getDegrees() with disconnected data
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, NullValue, Long> graph =
+                Graph.fromDataSet(TestGraphUtils.getDisconnectedLongLongEdgeData(env), env);
+
+        DataSet<Tuple2<Long,Long>> data =graph.outDegrees();
+        List<Tuple2<Long,Long>> result= data.collect();
+        
+        expectedResult = "1,2\n" +
+                "2,1\n" +
+                "3,0\n" +
+                "4,1\n" +
+                "5,0\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
new file mode 100644
index 0000000..955122f
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -0,0 +1,189 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class DegreesWithExceptionITCase {
+
+	private static final int PARALLELISM = 4;
+
+	private static ForkableFlinkMiniCluster cluster;
+	
+
+	@BeforeClass
+	public static void setupCluster() {
+		try {
+			Configuration config = new Configuration();
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail("Error starting test cluster: " + e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDownCluster() {
+		try {
+			cluster.stop();
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Cluster shutdown caused an exception: " + t.getMessage());
+		}
+	}
+
+	/**
+	 * Test outDegrees() with an edge having a srcId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testOutDegreesInvalidEdgeSrcId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
+
+		try {
+			graph.outDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+
+			fail("graph.outDegrees() did not fail.");
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test inDegrees() with an edge having a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testInDegreesInvalidEdgeTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
+
+		try {
+			graph.inDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+
+			fail("graph.inDegrees() did not fail.");
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test getDegrees() with an edge having a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGetDegreesInvalidEdgeTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
+
+		try {
+			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+
+			fail("graph.getDegrees() did not fail.");
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test getDegrees() with an edge having a srcId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGetDegreesInvalidEdgeSrcId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
+
+		try {
+			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+
+			fail("graph.getDegrees() did not fail.");
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test getDegrees() with an edge having a srcId and a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGetDegreesInvalidEdgeSrcTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcTrgData(env), env);
+
+		try {
+			graph.getDegrees().output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+
+			fail("graph.getDegrees() did not fail.");
+		}
+		catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
new file mode 100644
index 0000000..5a64dd7
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/FromCollectionITCase.java
@@ -0,0 +1,118 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+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.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class FromCollectionITCase extends MultipleProgramsTestBase {
+
+	public FromCollectionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+
+	@Test
+	public void testFromCollectionVerticesEdges() throws Exception {
+		/*
+		 * Test fromCollection(vertices, edges):
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongVertices(),
+                TestGraphUtils.getLongLongEdges(), env);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+        
+        expectedResult = "1,2,12\n" +
+	                "1,3,13\n" +
+	                "2,3,23\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testFromCollectionEdgesNoInitialValue() throws Exception {
+        /*
+         * Test fromCollection(edges) with no initial value for the vertices
+         */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+        Graph<Long, NullValue, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(),
+        		env);
+
+        
+        DataSet<Vertex<Long,NullValue>> data = graph.getVertices();
+        List<Vertex<Long,NullValue>> result= data.collect();
+        
+        expectedResult = "1,(null)\n" +
+	                "2,(null)\n" +
+	                "3,(null)\n" +
+	                "4,(null)\n" +
+	                "5,(null)\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testFromCollectionEdgesWithInitialValue() throws Exception {
+        /*
+         * Test fromCollection(edges) with vertices initialised by a
+         * function that takes the id and doubles it
+         */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromCollection(TestGraphUtils.getLongLongEdges(),
+                new InitVerticesMapper(), env);
+
+        DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+        
+        expectedResult = "1,2\n" +
+	                "2,4\n" +
+	                "3,6\n" +
+	                "4,8\n" +
+	                "5,10\n";
+        
+        compareResultAsTuples(result, expectedResult);
+    }
+
+	@SuppressWarnings("serial")
+	private static final class InitVerticesMapper implements MapFunction<Long, Long> {
+        public Long map(Long vertexId) {
+            return vertexId * 2;
+        }
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.java
new file mode 100644
index 0000000..22a5151
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationITCase.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.test.operations;
+
+import java.util.LinkedList;
+import java.util.List;
+
+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.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
+import org.apache.flink.graph.validation.InvalidVertexIdsValidator;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class GraphCreationITCase extends MultipleProgramsTestBase {
+
+	public GraphCreationITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+
+    private String expectedResult;
+
+	@Test
+	public void testCreateWithoutVertexValues() throws Exception {
+	/*
+	 * Test create() with edge dataset and no vertex values
+     */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, NullValue, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Vertex<Long,NullValue>> data = graph.getVertices();
+        List<Vertex<Long,NullValue>> result= data.collect();
+        
+		expectedResult = "1,(null)\n" +
+					"2,(null)\n" +
+					"3,(null)\n" +
+					"4,(null)\n" +
+					"5,(null)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCreateWithMapper() throws Exception {
+	/*
+	 * Test create() with edge dataset and a mapper that assigns the id as value
+     */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env),
+				new AssignIdAsValueMapper(), env);
+
+        DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+        
+		expectedResult = "1,1\n" +
+					"2,2\n" +
+					"3,3\n" +
+					"4,4\n" +
+					"5,5\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCreateWithCustomVertexValue() throws Exception {
+		/*
+		 * Test create() with edge dataset and a mapper that assigns a parametrized custom vertex value
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, DummyCustomParameterizedType<Double>, Long> graph = Graph.fromDataSet(
+				TestGraphUtils.getLongLongEdgeData(env), new AssignCustomVertexValueMapper(), env);
+
+        DataSet<Vertex<Long,DummyCustomParameterizedType<Double>>> data = graph.getVertices();
+        List<Vertex<Long,DummyCustomParameterizedType<Double>>> result= data.collect();
+        
+		expectedResult = "1,(2.0,0)\n" +
+				"2,(4.0,1)\n" +
+				"3,(6.0,2)\n" +
+				"4,(8.0,3)\n" +
+				"5,(10.0,4)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testValidate() throws Exception {
+		/*
+		 * Test validate():
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Vertex<Long, Long>> vertices = TestGraphUtils.getLongLongVertexData(env);
+		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
+		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
+
+		//env.fromElements(result).writeAsText(resultPath);
+		
+		String res= valid.toString();//env.fromElements(valid);
+        List<String> result= new LinkedList<String>();
+        result.add(res);
+		expectedResult = "true";
+		
+		compareResultAsText(result, expectedResult);
+	}
+
+	@Test
+	public void testValidateWithInvalidIds() throws Exception {
+		/*
+		 * Test validate() - invalid vertex ids
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Vertex<Long, Long>> vertices = TestGraphUtils.getLongLongInvalidVertexData(env);
+		DataSet<Edge<Long, Long>> edges = TestGraphUtils.getLongLongEdgeData(env);
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
+		Boolean valid = graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>());
+		
+		String res= valid.toString();//env.fromElements(valid);
+        List<String> result= new LinkedList<String>();
+        result.add(res);
+
+		expectedResult = "false\n";
+		
+		compareResultAsText(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignIdAsValueMapper implements MapFunction<Long, Long> {
+		public Long map(Long vertexId) {
+			return vertexId;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignCustomVertexValueMapper implements
+		MapFunction<Long, DummyCustomParameterizedType<Double>> {
+
+		DummyCustomParameterizedType<Double> dummyValue =
+				new DummyCustomParameterizedType<Double>();
+
+		public DummyCustomParameterizedType<Double> map(Long vertexId) {
+			dummyValue.setIntField(vertexId.intValue()-1);
+			dummyValue.setTField(vertexId*2.0);
+			return dummyValue;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
new file mode 100644
index 0000000..99c66ec
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithCsvITCase.java
@@ -0,0 +1,204 @@
+/*
+ * 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.operations;
+
+import com.google.common.base.Charsets;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Triplet;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.NullValue;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class GraphCreationWithCsvITCase extends MultipleProgramsTestBase {
+
+	public GraphCreationWithCsvITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private String expectedResult;
+
+	@Test
+	public void testCreateWithCsvFile() throws Exception {
+		/*
+		 * Test with two Csv files one with Vertex Data and one with Edges data
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final String fileContent =  "1,1\n"+
+				"2,2\n"+
+				"3,3\n";
+		final FileInputSplit split = createTempFile(fileContent);
+		final String fileContent2 =  "1,2,ot\n"+
+				"3,2,tt\n"+
+				"3,1,to\n";
+		final FileInputSplit split2 = createTempFile(fileContent2);
+
+		Graph<Long, Long, String> graph = Graph.fromCsvReader(split.getPath().toString(),split2.getPath().toString(),env)
+				.types(Long.class, Long.class, String.class);
+
+		List<Triplet<Long, Long, String>> result = graph.getTriplets().collect();
+
+		expectedResult = "1,2,1,2,ot\n" +
+				"3,2,3,2,tt\n" +
+				"3,1,3,1,to\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCsvWithNullEdge() throws Exception {
+		/*
+		Test fromCsvReader with edge and vertex path and nullvalue for edge
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final String vertexFileContent = "1,one\n"+
+				"2,two\n"+
+				"3,three\n";
+		final String edgeFileContent = "1,2\n"+
+				"3,2\n"+
+				"3,1\n";
+		final FileInputSplit split = createTempFile(vertexFileContent);
+		final FileInputSplit edgeSplit = createTempFile(edgeFileContent);
+
+		Graph<Long, String, NullValue> graph = Graph.fromCsvReader(split.getPath().toString(), edgeSplit.getPath().toString(),
+				env).vertexTypes(Long.class, String.class);
+
+		List<Triplet<Long, String, NullValue>> result = graph.getTriplets().collect();
+
+		expectedResult = "1,2,one,two,(null)\n"+
+				"3,2,three,two,(null)\n"+
+				"3,1,three,one,(null)\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCsvWithConstantValueMapper() throws Exception {
+		/*
+		*Test fromCsvReader with edge path and a mapper that assigns a Double constant as value
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final String fileContent =  "1,2,ot\n"+
+				"3,2,tt\n"+
+				"3,1,to\n";
+		final FileInputSplit split = createTempFile(fileContent);
+
+		Graph<Long, Double, String> graph = Graph.fromCsvReader(split.getPath().toString(),
+				new AssignDoubleValueMapper(), env).types(Long.class, Double.class, String.class);
+
+		List<Triplet<Long, Double, String>> result = graph.getTriplets().collect();
+		//graph.getTriplets().writeAsCsv(resultPath);
+		expectedResult = "1,2,0.1,0.1,ot\n" + "3,1,0.1,0.1,to\n" + "3,2,0.1,0.1,tt\n";
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCreateWithOnlyEdgesCsvFile() throws Exception {
+		/*
+		 * Test with one Csv file one with Edges data. Also tests the configuration method ignoreFistLineEdges()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final String fileContent2 =  "header\n1,2,ot\n"+
+				"3,2,tt\n"+
+				"3,1,to\n";
+
+		final FileInputSplit split2 = createTempFile(fileContent2);
+		Graph<Long, NullValue, String> graph= Graph.fromCsvReader(split2.getPath().toString(), env)
+				.ignoreFirstLineEdges()
+				.ignoreCommentsVertices("hi")
+				.edgeTypes(Long.class, String.class);
+
+		List<Triplet<Long, NullValue, String>> result = graph.getTriplets().collect();
+		expectedResult = "1,2,(null),(null),ot\n" +
+				"3,2,(null),(null),tt\n" +
+				"3,1,(null),(null),to\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testCreateCsvFileDelimiterConfiguration() throws Exception {
+		/*
+		 * Test with an Edge and Vertex csv file. Tests the configuration methods FieldDelimiterEdges and
+		 * FieldDelimiterVertices
+		 * Also tests the configuration methods LineDelimiterEdges and LineDelimiterVertices
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		final String fileContent =  "header\n1;1\n"+
+				"2;2\n"+
+				"3;3\n";
+
+		final FileInputSplit split = createTempFile(fileContent);
+
+		final String fileContent2 =  "header|1:2:ot|"+
+				"3:2:tt|"+
+				"3:1:to|";
+
+		final FileInputSplit split2 = createTempFile(fileContent2);
+
+		Graph<Long, Long, String> graph= Graph.fromCsvReader(split.getPath().toString(),split2.getPath().toString(),env).
+				ignoreFirstLineEdges().ignoreFirstLineVertices().
+				fieldDelimiterEdges(":").fieldDelimiterVertices(";").
+				lineDelimiterEdges("|").
+				types(Long.class, Long.class, String.class);
+
+		List<Triplet<Long, Long, String>> result = graph.getTriplets().collect();
+
+		expectedResult = "1,2,1,2,ot\n" +
+				"3,2,3,2,tt\n" +
+				"3,1,3,1,to\n";
+
+		compareResultAsTuples(result, expectedResult);
+
+	}
+
+	/*----------------------------------------------------------------------------------------------------------------*/
+	@SuppressWarnings("serial")
+	private static final class AssignDoubleValueMapper implements MapFunction<Long, Double> {
+		public Double map(Long value) {
+			return 0.1d;
+		}
+	}
+
+	private FileInputSplit createTempFile(String content) throws IOException {
+		File tempFile = File.createTempFile("test_contents", "tmp");
+		tempFile.deleteOnExit();
+
+		OutputStreamWriter wrt = new OutputStreamWriter(
+				new FileOutputStream(tempFile), Charsets.UTF_8
+		);
+		wrt.write(content);
+		wrt.close();
+
+		return new FileInputSplit(0, new Path(tempFile.toURI().toString()), 0,
+							tempFile.length(), new String[] {"localhost"});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
new file mode 100644
index 0000000..20cbca5
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphCreationWithMapperITCase.java
@@ -0,0 +1,158 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class GraphCreationWithMapperITCase extends MultipleProgramsTestBase {
+
+	public GraphCreationWithMapperITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+
+	@Test
+	public void testWithDoubleValueMapper() throws Exception {
+		/*
+		 * Test create() with edge dataset and a mapper that assigns a double constant as value
+	     */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Double, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongEdgeData(env),
+				new AssignDoubleValueMapper(), env);
+
+        DataSet<Vertex<Long,Double>> data = graph.getVertices();
+        List<Vertex<Long,Double>> result= data.collect();
+		
+		expectedResult = "1,0.1\n" +
+				"2,0.1\n" +
+				"3,0.1\n" +
+				"4,0.1\n" +
+				"5,0.1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithTuple2ValueMapper() throws Exception {
+		/*
+		 * Test create() with edge dataset and a mapper that assigns a Tuple2 as value
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Tuple2<Long, Long>, Long> graph = Graph.fromDataSet(
+				TestGraphUtils.getLongLongEdgeData(env), new AssignTuple2ValueMapper(), env);
+
+        DataSet<Vertex<Long, Tuple2<Long, Long>>> data = graph.getVertices();
+        List<Vertex<Long, Tuple2<Long, Long>>> result= data.collect();
+        
+		expectedResult = "1,(2,42)\n" +
+				"2,(4,42)\n" +
+				"3,(6,42)\n" +
+				"4,(8,42)\n" +
+				"5,(10,42)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithConstantValueMapper() throws Exception {
+	/*
+	 * Test create() with edge dataset with String key type
+	 * and a mapper that assigns a double constant as value
+	 */
+	final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+	Graph<String, Double, Long> graph = Graph.fromDataSet(TestGraphUtils.getStringLongEdgeData(env),
+			new AssignDoubleConstantMapper(), env);
+
+    DataSet<Vertex<String,Double>> data = graph.getVertices();
+    List<Vertex<String,Double>> result= data.collect();
+    
+	expectedResult = "1,0.1\n" +
+			"2,0.1\n" +
+			"3,0.1\n" +
+			"4,0.1\n" +
+			"5,0.1\n";
+	
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithDCustomValueMapper() throws Exception {
+		/*
+		 * Test create() with edge dataset and a mapper that assigns a custom vertex value
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, DummyCustomType, Long> graph = Graph.fromDataSet(
+				TestGraphUtils.getLongLongEdgeData(env), new AssignCustomValueMapper(), env);
+
+	    DataSet<Vertex<Long,DummyCustomType>> data = graph.getVertices();
+	    List<Vertex<Long,DummyCustomType>> result= data.collect();
+	    
+		expectedResult = "1,(F,0)\n" +
+				"2,(F,1)\n" +
+				"3,(F,2)\n" +
+				"4,(F,3)\n" +
+				"5,(F,4)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignDoubleValueMapper implements MapFunction<Long, Double> {
+		public Double map(Long value) {
+			return 0.1d;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignTuple2ValueMapper implements MapFunction<Long, Tuple2<Long, Long>> {
+		public Tuple2<Long, Long> map(Long vertexId) {
+			return new Tuple2<Long, Long>(vertexId*2, 42l);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignDoubleConstantMapper implements MapFunction<String, Double> {
+		public Double map(String value) {
+			return 0.1d;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AssignCustomValueMapper implements MapFunction<Long, DummyCustomType> {
+		public DummyCustomType map(Long vertexId) {
+			return new DummyCustomType(vertexId.intValue()-1, false);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
new file mode 100644
index 0000000..d6e5a9c
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphMutationsITCase.java
@@ -0,0 +1,603 @@
+/*
+ * 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.operations;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class GraphMutationsITCase extends MultipleProgramsTestBase {
+
+	public GraphMutationsITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+
+	@Test
+	public void testAddVertex() throws Exception {
+		/*
+		 * Test addVertex() -- simple case
+		 */	
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		graph = graph.addVertex(new Vertex<Long, Long>(6L, 6L));
+        
+		DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n" +
+				"6,6\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddVertices() throws Exception {
+		/*
+		 * Test addVertices() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(6L, 6L));
+		vertices.add(new Vertex<Long, Long>(7L, 7L));
+
+		graph = graph.addVertices(vertices);
+
+		DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n" +
+				"6,6\n" +
+				"7,7\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddVertexExisting() throws Exception {
+		/*
+		 * Test addVertex() -- add an existing vertex
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		graph = graph.addVertex(new Vertex<Long, Long>(1L, 1L));
+		
+		DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddVerticesBothExisting() throws Exception {
+		/*
+		 * Test addVertices() -- add two existing vertices
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(1L, 1L));
+		vertices.add(new Vertex<Long, Long>(3L, 3L));
+
+		graph = graph.addVertices(vertices);
+
+		DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddVerticesOneExisting() throws Exception {
+		/*
+		 * Test addVertices() -- add an existing vertex
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		vertices.add(new Vertex<Long, Long>(1L, 1L));
+		vertices.add(new Vertex<Long, Long>(6L, 6L));
+
+		graph = graph.addVertices(vertices);
+
+		DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n" +
+				"6,6\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveVertex() throws Exception {
+		/*
+		 * Test removeVertex() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.removeVertex(new Vertex<Long, Long>(5L, 5L));
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveVertices() throws Exception {
+		/*
+		 * Test removeVertices() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
+		verticesToBeRemoved.add(new Vertex<Long, Long>(1L, 1L));
+		verticesToBeRemoved.add(new Vertex<Long, Long>(2L, 2L));
+
+		graph = graph.removeVertices(verticesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveInvalidVertex() throws Exception {
+		/*
+		 * Test removeVertex() -- remove an invalid vertex
+		 */	
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.removeVertex(new Vertex<Long, Long>(6L, 6L));
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveOneValidOneInvalidVertex() throws Exception {
+		/*
+		 * Test removeVertices() -- remove one invalid vertex and a valid one
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
+		verticesToBeRemoved.add(new Vertex<Long, Long>(1L, 1L));
+		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
+
+		graph = graph.removeVertices(verticesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveBothInvalidVertices() throws Exception {
+		/*
+		 * Test removeVertices() -- remove two invalid vertices
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
+		verticesToBeRemoved.add(new Vertex<Long, Long>(6L, 6L));
+		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
+
+		graph = graph.removeVertices(verticesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveBothInvalidVerticesVertexResult() throws Exception {
+		/*
+		 * Test removeVertices() -- remove two invalid vertices and verify the data set of vertices
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Vertex<Long, Long>> verticesToBeRemoved = new ArrayList<Vertex<Long, Long>>();
+		verticesToBeRemoved.add(new Vertex<Long, Long>(6L, 6L));
+		verticesToBeRemoved.add(new Vertex<Long, Long>(7L, 7L));
+
+		graph = graph.removeVertices(verticesToBeRemoved);
+
+        DataSet<Vertex<Long,Long>> data = graph.getVertices();
+        List<Vertex<Long, Long>> result= data.collect();
+
+		expectedResult = "1,1\n" +
+				"2,2\n" +
+				"3,3\n" +
+				"4,4\n" +
+				"5,5\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+	
+	@Test
+	public void testAddEdge() throws Exception {
+		/*
+		 * Test addEdge() -- simple case
+		 */
+		
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.addEdge(new Vertex<Long, Long>(6L, 6L), new Vertex<Long, Long>(1L, 1L),
+				61L);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n" +
+				"6,1,61\n";	
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddEdges() throws Exception {
+		/*
+		 * Test addEdges() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Edge<Long, Long>> edgesToBeAdded = new ArrayList<Edge<Long, Long>>();
+		edgesToBeAdded.add(new Edge<Long, Long>(2L, 4L, 24L));
+		edgesToBeAdded.add(new Edge<Long, Long>(4L, 1L, 41L));
+
+		graph = graph.addEdges(edgesToBeAdded);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"2,4,24\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,1,41\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddEdgesInvalidVertices() throws Exception {
+		/*
+		 * Test addEdges() -- the source and target vertices do not exist in the graph
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Edge<Long, Long>> edgesToBeAdded = new ArrayList<Edge<Long, Long>>();
+		edgesToBeAdded.add(new Edge<Long, Long>(6L, 1L, 61L));
+		edgesToBeAdded.add(new Edge<Long, Long>(7L, 1L, 71L));
+
+		graph = graph.addEdges(edgesToBeAdded);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAddExistingEdge() throws Exception {
+		/*
+		 * Test addEdge() -- add already existing edge
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.addEdge(new Vertex<Long, Long>(1L, 1L), new Vertex<Long, Long>(2L, 2L),
+				12L);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";	
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveEdge() throws Exception {
+		/*
+		 * Test removeEdge() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.removeEdge(new Edge<Long, Long>(5L, 1L, 51L));
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveEdges() throws Exception {
+		/*
+		 * Test removeEdges() -- simple case
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
+		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
+		edgesToBeRemoved.add(new Edge<Long, Long>(2L, 3L, 23L));
+
+		graph = graph.removeEdges(edgesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveSameEdgeTwice() throws Exception {
+		/*
+		 * Test removeEdges() -- try to remove the same edge twice
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
+		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
+		edgesToBeRemoved.add(new Edge<Long, Long>(5L, 1L, 51L));
+
+		graph = graph.removeEdges(edgesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveInvalidEdge() throws Exception {
+		/*
+		 * Test removeEdge() -- invalid edge
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		graph = graph.removeEdge(new Edge<Long, Long>(6L, 1L, 61L));
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testRemoveOneValidOneInvalidEdge() throws Exception {
+		/*
+		 * Test removeEdges() -- one edge is valid, the other is invalid
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		List<Edge<Long, Long>> edgesToBeRemoved = new ArrayList<Edge<Long, Long>>();
+		edgesToBeRemoved.add(new Edge<Long, Long>(1L, 1L, 51L));
+		edgesToBeRemoved.add(new Edge<Long, Long>(6L, 1L, 61L));
+
+		graph = graph.removeEdges(edgesToBeRemoved);
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+}
\ No newline at end of file


[03/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
deleted file mode 100644
index ffc9da9..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
+++ /dev/null
@@ -1,378 +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.operations;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-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.Triplet;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class GraphOperationsITCase extends MultipleProgramsTestBase {
-
-	public GraphOperationsITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testUndirected() throws Exception {
-		/*
-		 * Test getUndirected()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Edge<Long,Long>> data = graph.getUndirected().getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-        
-		expectedResult = "1,2,12\n" + "2,1,12\n" +
-					"1,3,13\n" + "3,1,13\n" +
-					"2,3,23\n" + "3,2,23\n" +
-					"3,4,34\n" + "4,3,34\n" +
-					"3,5,35\n" + "5,3,35\n" +
-					"4,5,45\n" + "5,4,45\n" +
-					"5,1,51\n" + "1,5,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testReverse() throws Exception {
-		/*
-		 * Test reverse()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Edge<Long,Long>> data = graph.reverse().getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-        
-		expectedResult = "2,1,12\n" +
-					"3,1,13\n" +
-					"3,2,23\n" +
-					"4,3,34\n" +
-					"5,3,35\n" +
-					"5,4,45\n" +
-					"1,5,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	@Test
-	public void testSubGraph() throws Exception {
-		/*
-		 * Test subgraph:
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long,Long>> data= graph.subgraph(new FilterFunction<Vertex<Long, Long>>() {
-						   public boolean filter(Vertex<Long, Long> vertex) throws Exception {
-							   return (vertex.getValue() > 2);
-						   }
-					   },
-				new FilterFunction<Edge<Long, Long>>() {
-					public boolean filter(Edge<Long, Long> edge) throws Exception {
-						return (edge.getValue() > 34);
-					}
-				}).getEdges();
-
-        List<Edge<Long, Long>> result= data.collect();
-        
-		expectedResult = "3,5,35\n" +
-					"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	@Test
-	public void testFilterVertices() throws Exception {
-		/*
-		 * Test filterOnVertices:
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long,Long>> data = graph.filterOnVertices(new FilterFunction<Vertex<Long, Long>>() {
-			public boolean filter(Vertex<Long, Long> vertex) throws Exception {
-				return (vertex.getValue() > 2);
-			}
-		}).getEdges();
-
-        List<Edge<Long, Long>> result= data.collect();
-		
-		expectedResult =  "3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	@Test
-	public void testFilterEdges() throws Exception {
-		/*
-		 * Test filterOnEdges:
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long,Long>> data = graph.filterOnEdges(new FilterFunction<Edge<Long, Long>>() {
-			public boolean filter(Edge<Long, Long> edge) throws Exception {
-				return (edge.getValue() > 34);
-			}
-		}).getEdges();
-
-        List<Edge<Long, Long>> result = data.collect();
-        
-		expectedResult = "3,5,35\n" +
-					"4,5,45\n" +
-					"5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testNumberOfVertices() throws Exception {
-		/*
-		 * Test numberOfVertices()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		DataSet<Long> data = env.fromElements(graph.numberOfVertices());
-
-        List<Long> result= data.collect();
-        
-		expectedResult = "5";
-		
-		compareResultAsText(result, expectedResult);
-	}
-
-	@Test
-	public void testNumberOfEdges() throws Exception {
-		/*
-		 * Test numberOfEdges()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		DataSet<Long> data = env.fromElements(graph.numberOfEdges());
-
-        List<Long> result= data.collect();
-        
-		expectedResult = "7";
-		
-		compareResultAsText(result, expectedResult);
-	}
-
-	@Test
-	public void testVertexIds() throws Exception {
-		/*
-		 * Test getVertexIds()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Long> data = graph.getVertexIds();
-        List<Long> result= data.collect();
-        
-		expectedResult = "1\n2\n3\n4\n5\n";
-		
-		compareResultAsText(result, expectedResult);
-	}
-
-	@Test
-	public void testEdgesIds() throws Exception {
-		/*
-		 * Test getEdgeIds()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Tuple2<Long,Long>> data = graph.getEdgeIds();
-        List<Tuple2<Long, Long>> result= data.collect();
-        
-		expectedResult = "1,2\n" + "1,3\n" +
-				"2,3\n" + "3,4\n" +
-				"3,5\n" + "4,5\n" +
-				"5,1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testUnion() throws Exception {
-		/*
-		 * Test union()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
-		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
-
-		vertices.add(new Vertex<Long, Long>(6L, 6L));
-		edges.add(new Edge<Long, Long>(6L, 1L, 61L));
-
-		graph = graph.union(Graph.fromCollection(vertices, edges, env));
-
-        DataSet<Edge<Long,Long>> data = graph.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-		expectedResult = "1,2,12\n" +
-					"1,3,13\n" +
-					"2,3,23\n" +
-					"3,4,34\n" +
-					"3,5,35\n" +
-					"4,5,45\n" +
-					"5,1,51\n" +
-					"6,1,61\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testDifference() throws Exception {
-		/*Test  difference() method  by checking    the output  for getEdges()   on  the resultant   graph
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(TestGraphUtils.getLongLongVertexDataDifference(env),
-				TestGraphUtils.getLongLongEdgeDataDifference(env), env);
-
-		graph = graph.difference(graph2);
-
-		List<Edge<Long, Long>> result = graph.getEdges().collect();
-
-		expectedResult = "4,5,45\n";
-		compareResultAsTuples(result, expectedResult);
-	}
-
-
-	@Test
-	public void testDifferenceVertices() throws Exception{
-		/*Test  difference() method  by checking    the output  for getVertices()   on  the resultant   graph
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(TestGraphUtils.getLongLongVertexDataDifference(env),
-				TestGraphUtils.getLongLongEdgeDataDifference(env), env);
-
-		graph = graph.difference(graph2);
-
-		List<Vertex<Long, Long>> result = graph.getVertices().collect();
-
-		expectedResult =  "2,2\n" +
-				"4,4\n" +
-				"5,5\n" ;
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testDifference2() throws Exception {
-		/*
-		 * Test difference() such that no common vertices are there
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Vertex<Long, Long>> vertex = env.fromElements(new Vertex<Long, Long>(6L, 6L));
-
-		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(vertex,TestGraphUtils.getLongLongEdgeDataDifference2(env),env);
-
-		graph = graph.difference(graph2);
-
-		List<Edge<Long, Long>> result = graph.getEdges().collect();
-
-		expectedResult =	"1,2,12\n" +
-				"1,3,13\n" +
-				"2,3,23\n" +
-				"3,4,34\n" +
-				"3,5,35\n" +
-				"4,5,45\n" +
-				"5,1,51\n" ;
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testTriplets() throws Exception {
-		/*
-		 * Test getTriplets()
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-        DataSet<Triplet<Long,Long,Long>> data = graph.getTriplets();
-        List<Triplet<Long,Long,Long>> result= data.collect();
-
-		expectedResult = "1,2,1,2,12\n" + "1,3,1,3,13\n" +
-				"2,3,2,3,23\n" + "3,4,3,4,34\n" +
-				"3,5,3,5,35\n" + "4,5,4,5,45\n" +
-				"5,1,5,1,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
deleted file mode 100644
index e406ce2..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
+++ /dev/null
@@ -1,532 +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.operations;
-
-import java.util.List;
-
-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.api.java.tuple.Tuple3;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
-import org.apache.flink.graph.utils.EdgeToTuple3Map;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
-
-	public JoinWithEdgesITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testWithEdgesInputDataset() throws Exception {
-		/*
-		 * Test joinWithEdges with the input DataSet parameter identical
-		 * to the edge DataSet
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges()
-                        .map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,68\n" +
-	                "3,5,70\n" +
-	                "4,5,90\n" +
-	                "5,1,102\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithLessElements() throws Exception {
-	    /*
-		 * Test joinWithEdges with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet, but of the same type
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges().first(3)
-                        .map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithLessElementsDifferentType() throws Exception {
-	    /*
-		 * Test joinWithEdges with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet and of a different type(Boolean)
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges().first(3)
-                        .map(new BooleanEdgeValueMapper()), new DoubleIfTrueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithNoCommonKeys() throws Exception {
-	    /*
-		 * Test joinWithEdges with the input DataSet containing different keys than the edge DataSet
-		 * - the iterator becomes empty.
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdges(TestGraphUtils.getLongLongLongTuple3Data(env),
-                new DoubleValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,68\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithCustomType() throws Exception {
-	    /*
-	     * Test joinWithEdges with a DataSet containing custom parametrised type input values
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdges(TestGraphUtils.getLongLongCustomTuple3Data(env),
-                new CustomValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,10\n" +
-	                "1,3,20\n" +
-	                "2,3,30\n" +
-	                "3,4,40\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithEdgesOnSource() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnSource with the input DataSet parameter identical
-		 * to the edge DataSet
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges()
-                        .map(new ProjectSourceAndValueMapper()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,25\n" +
-	                "2,3,46\n" +
-	                "3,4,68\n" +
-	                "3,5,69\n" +
-	                "4,5,90\n" +
-	                "5,1,102\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnSourceWithLessElements() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet, but of the same type
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges().first(3)
-                        .map(new ProjectSourceAndValueMapper()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,25\n" +
-	                "2,3,46\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnSourceWithDifferentType() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet and of a different type(Boolean)
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges().first(3)
-                        .map(new ProjectSourceWithTrueMapper()), new DoubleIfTrueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnSourceWithNoCommonKeys() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnSource with the input DataSet containing different keys than the edge DataSet
-		 * - the iterator becomes empty.
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(TestGraphUtils.getLongLongTuple2SourceData(env),
-                new DoubleValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,20\n" +
-	                "1,3,20\n" +
-	                "2,3,60\n" +
-	                "3,4,80\n" +
-	                "3,5,80\n" +
-	                "4,5,120\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnSourceWithCustom() throws Exception {
-	    /*
-	     * Test joinWithEdgesOnSource with a DataSet containing custom parametrised type input values
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(TestGraphUtils.getLongCustomTuple2SourceData(env),
-                new CustomValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,10\n" +
-	                "1,3,10\n" +
-	                "2,3,30\n" +
-	                "3,4,40\n" +
-	                "3,5,40\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithEdgesOnTarget() throws Exception {
-    /*
-	 * Test joinWithEdgesOnTarget with the input DataSet parameter identical
-	 * to the edge DataSet
-	 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges()
-                        .map(new ProjectTargetAndValueMapper()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,36\n" +
-	                "3,4,68\n" +
-	                "3,5,70\n" +
-	                "4,5,80\n" +
-	                "5,1,102\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithOnTargetWithLessElements() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet, but of the same type
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges().first(3)
-                        .map(new ProjectTargetAndValueMapper()), new AddValuesMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,36\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnTargetWithDifferentType() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing
-		 * less elements than the edge DataSet and of a different type(Boolean)
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges().first(3)
-                        .map(new ProjectTargetWithTrueMapper()), new DoubleIfTrueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,24\n" +
-	                "1,3,26\n" +
-	                "2,3,46\n" +
-	                "3,4,34\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnTargetWithNoCommonKeys() throws Exception {
-	    /*
-		 * Test joinWithEdgesOnTarget with the input DataSet containing different keys than the edge DataSet
-		 * - the iterator becomes empty.
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongLongTuple2TargetData(env),
-                new DoubleValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,20\n" +
-	                "1,3,40\n" +
-	                "2,3,40\n" +
-	                "3,4,80\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,140\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testOnTargetWithCustom() throws Exception {
-	    /*
-	     * Test joinWithEdgesOnTarget with a DataSet containing custom parametrised type input values
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongCustomTuple2TargetData(env),
-                new CustomValueMapper());
-
-        DataSet<Edge<Long,Long>> data = res.getEdges();
-        List<Edge<Long, Long>> result= data.collect();
-
-        expectedResult = "1,2,10\n" +
-	                "1,3,20\n" +
-	                "2,3,20\n" +
-	                "3,4,40\n" +
-	                "3,5,35\n" +
-	                "4,5,45\n" +
-	                "5,1,51\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@SuppressWarnings("serial")
-	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-		public Long map(Tuple2<Long, Long> tuple) throws Exception {
-			return tuple.f0 + tuple.f1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class BooleanEdgeValueMapper implements MapFunction<Edge<Long, Long>, Tuple3<Long, Long, Boolean>> {
-        public Tuple3<Long, Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
-            return new Tuple3<Long, Long, Boolean>(edge.getSource(),
-                    edge.getTarget(), true);
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
-        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
-            if(tuple.f1) {
-                return tuple.f0 * 2;
-            }
-            else {
-                return tuple.f0;
-            }
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class DoubleValueMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-        public Long map(Tuple2<Long, Long> tuple) throws Exception {
-            return tuple.f1 * 2;
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
-        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
-            return (long) tuple.f1.getIntField();
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class ProjectSourceAndValueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Long>> {
-        public Tuple2<Long, Long> map(Edge<Long, Long> edge) throws Exception {
-            return new Tuple2<Long, Long>(edge.getSource(), edge.getValue());
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class ProjectSourceWithTrueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Boolean>> {
-        public Tuple2<Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
-            return new Tuple2<Long, Boolean>(edge.getSource(), true);
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class ProjectTargetAndValueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Long>> {
-        public Tuple2<Long, Long> map(Edge<Long, Long> edge) throws Exception {
-            return new Tuple2<Long, Long>(edge.getTarget(), edge.getValue());
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class ProjectTargetWithTrueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Boolean>> {
-        public Tuple2<Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
-            return new Tuple2<Long, Boolean>(edge.getTarget(), true);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
deleted file mode 100644
index 22a5535..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
+++ /dev/null
@@ -1,214 +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.operations;
-
-import java.util.List;
-
-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.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
-import org.apache.flink.graph.utils.VertexToTuple2Map;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
-
-	public JoinWithVerticesITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testJoinWithVertexSet() throws Exception {
-		/*
-		 * Test joinWithVertices with the input DataSet parameter identical
-		 * to the vertex DataSet
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices()
-                        .map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-       expectedResult = "1,2\n" +
-	                "2,4\n" +
-	                "3,6\n" +
-	                "4,8\n" +
-	                "5,10\n";
-       
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithLessElements() throws Exception {
-	/*
-	 * Test joinWithVertices with the input DataSet passed as a parameter containing
-	 * less elements than the vertex DataSet, but of the same type
-	 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices().first(3)
-                        .map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-        expectedResult = "1,2\n" +
-	                "2,4\n" +
-	                "3,6\n" +
-	                "4,4\n" +
-	                "5,5\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithDifferentType() throws Exception {
-	/*
-	 * Test joinWithVertices with the input DataSet passed as a parameter containing
-	 * less elements than the vertex DataSet and of a different type(Boolean)
-	 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices().first(3)
-                        .map(new ProjectIdWithTrue()), new DoubleIfTrueMapper());
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-        expectedResult = "1,2\n" +
-	                "2,4\n" +
-	                "3,6\n" +
-	                "4,4\n" +
-	                "5,5\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithDifferentKeys() throws Exception {
-		/*
-		 * Test joinWithVertices with an input DataSet containing different keys than the vertex DataSet
-		 * - the iterator becomes empty.
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithVertices(TestGraphUtils.getLongLongTuple2Data(env),
-                new ProjectSecondMapper());
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-        expectedResult = "1,10\n" +
-	                "2,20\n" +
-	                "3,30\n" +
-	                "4,40\n" +
-	                "5,5\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@Test
-	public void testWithCustomType() throws Exception {
-		/*
-		 * Test joinWithVertices with a DataSet containing custom parametrised type input values
-		 */
-        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-                TestGraphUtils.getLongLongEdgeData(env), env);
-
-        Graph<Long, Long, Long> res = graph.joinWithVertices(TestGraphUtils.getLongCustomTuple2Data(env),
-                new CustomValueMapper());
-
-		DataSet<Vertex<Long,Long>> data = res.getVertices();
-        List<Vertex<Long,Long>> result= data.collect();
-
-        expectedResult = "1,10\n" +
-	                "2,20\n" +
-	                "3,30\n" +
-	                "4,40\n" +
-	                "5,5\n";
-        
-		compareResultAsTuples(result, expectedResult);
-    }
-
-	@SuppressWarnings("serial")
-	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-		public Long map(Tuple2<Long, Long> tuple) throws Exception {
-			return tuple.f0 + tuple.f1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ProjectIdWithTrue implements MapFunction<Vertex<Long, Long>, Tuple2<Long, Boolean>> {
-        public Tuple2<Long, Boolean> map(Vertex<Long, Long> vertex) throws Exception {
-            return new Tuple2<Long, Boolean>(vertex.getId(), true);
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
-        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
-            if(tuple.f1) {
-                return tuple.f0 * 2;
-            }
-            else {
-                return tuple.f0;
-            }
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class ProjectSecondMapper implements MapFunction<Tuple2<Long, Long>, Long> {
-        public Long map(Tuple2<Long, Long> tuple) throws Exception {
-            return tuple.f1;
-        }
-    }
-
-	@SuppressWarnings("serial")
-	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
-        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
-            return (long) tuple.f1.getIntField();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
deleted file mode 100644
index 35f7b0e..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
+++ /dev/null
@@ -1,210 +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.operations;
-
-import java.util.List;
-
-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.Tuple1;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class MapEdgesITCase extends MultipleProgramsTestBase {
-
-	public MapEdgesITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testWithSameValue() throws Exception {
-		/*
-		 * Test mapEdges() keeping the same value type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long, Long>> mappedEdges = graph.mapEdges(new AddOneMapper()).getEdges();
-        List<Edge<Long, Long>> result= mappedEdges.collect();
-        
-		expectedResult = "1,2,13\n" +
-				"1,3,14\n" +
-				"2,3,24\n" +
-				"3,4,35\n" +
-				"3,5,36\n" + 
-				"4,5,46\n" + 
-				"5,1,52\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithStringValue() throws Exception {
-		/*
-		 * Test mapEdges() and change the value type to String
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long, String>> mappedEdges = graph.mapEdges(new ToStringMapper()).getEdges();
-		List<Edge<Long, String>> result= mappedEdges.collect();
-		
-		expectedResult = "1,2,string(12)\n" +
-				"1,3,string(13)\n" +
-				"2,3,string(23)\n" +
-				"3,4,string(34)\n" +
-				"3,5,string(35)\n" + 
-				"4,5,string(45)\n" + 
-				"5,1,string(51)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithTuple1Type() throws Exception {
-		/*
-		 * Test mapEdges() and change the value type to a Tuple1
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long, Tuple1<Long>>> mappedEdges = graph.mapEdges(new ToTuple1Mapper()).getEdges();
-		List<Edge<Long, Tuple1<Long>>> result= mappedEdges.collect();
-
-		expectedResult = "1,2,(12)\n" +
-				"1,3,(13)\n" +
-				"2,3,(23)\n" +
-				"3,4,(34)\n" +
-				"3,5,(35)\n" + 
-				"4,5,(45)\n" + 
-				"5,1,(51)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithCustomType() throws Exception {
-		/*
-		 * Test mapEdges() and change the value type to a custom type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long, DummyCustomType>> mappedEdges = graph.mapEdges(new ToCustomTypeMapper()).getEdges();
-		List<Edge<Long, DummyCustomType>> result= mappedEdges.collect();
-
-		expectedResult = "1,2,(T,12)\n" +
-			"1,3,(T,13)\n" +
-			"2,3,(T,23)\n" +
-			"3,4,(T,34)\n" +
-			"3,5,(T,35)\n" + 
-			"4,5,(T,45)\n" + 
-			"5,1,(T,51)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithParametrizedCustomType() throws Exception {
-		/*
-		 * Test mapEdges() and change the value type to a parameterized custom type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Edge<Long, DummyCustomParameterizedType<Double>>> mappedEdges = graph.mapEdges(
-				new ToCustomParametrizedTypeMapper()).getEdges();
-		List<Edge<Long, DummyCustomParameterizedType<Double>>> result= mappedEdges.collect();
-	
-		expectedResult = "1,2,(12.0,12)\n" +
-			"1,3,(13.0,13)\n" +
-			"2,3,(23.0,23)\n" +
-			"3,4,(34.0,34)\n" +
-			"3,5,(35.0,35)\n" + 
-			"4,5,(45.0,45)\n" + 
-			"5,1,(51.0,51)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AddOneMapper implements MapFunction<Edge<Long, Long>, Long> {
-		public Long map(Edge<Long, Long> edge) throws Exception {
-			return edge.getValue()+1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToStringMapper implements MapFunction<Edge<Long, Long>, String> {
-		public String map(Edge<Long, Long> edge) throws Exception {
-			return String.format("string(%d)", edge.getValue());
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToTuple1Mapper implements MapFunction<Edge<Long, Long>, Tuple1<Long>> {
-		public Tuple1<Long> map(Edge<Long, Long> edge) throws Exception {
-			Tuple1<Long> tupleValue = new Tuple1<Long>();
-			tupleValue.setFields(edge.getValue());
-			return tupleValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToCustomTypeMapper implements MapFunction<Edge<Long, Long>, DummyCustomType> {
-		public DummyCustomType map(Edge<Long, Long> edge) throws Exception {
-			DummyCustomType dummyValue = new DummyCustomType();
-			dummyValue.setIntField(edge.getValue().intValue());						
-			return dummyValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToCustomParametrizedTypeMapper implements MapFunction<Edge<Long, Long>, 
-		DummyCustomParameterizedType<Double>> {
-
-		public DummyCustomParameterizedType<Double> map(Edge<Long, Long> edge) throws Exception {
-			DummyCustomParameterizedType<Double> dummyValue = new DummyCustomParameterizedType<Double>();
-			dummyValue.setIntField(edge.getValue().intValue());
-			dummyValue.setTField(new Double(edge.getValue()));						
-			return dummyValue;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
deleted file mode 100644
index 677a03c..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
+++ /dev/null
@@ -1,219 +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.operations;
-
-import java.util.List;
-
-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.Tuple1;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
-import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class MapVerticesITCase extends MultipleProgramsTestBase {
-
-	public MapVerticesITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testWithSameValue() throws Exception {
-		/*
-		 * Test mapVertices() keeping the same value type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Vertex<Long, Long>> mappedVertices = graph.mapVertices(new AddOneMapper()).getVertices();	
-        List<Vertex<Long, Long>> result= mappedVertices.collect();
-        
-		expectedResult = "1,2\n" +
-			"2,3\n" +
-			"3,4\n" +
-			"4,5\n" +
-			"5,6\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithStringValue() throws Exception {
-		/*
-		 * Test mapVertices() and change the value type to String
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Vertex<Long, String>> mappedVertices = graph.mapVertices(new ToStringMapper()).getVertices();
-        List<Vertex<Long, String>> result= mappedVertices.collect();
-
-		expectedResult = "1,one\n" +
-			"2,two\n" +
-			"3,three\n" +
-			"4,four\n" +
-			"5,five\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithtuple1Value() throws Exception {
-		/*
-		 * Test mapVertices() and change the value type to a Tuple1
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Vertex<Long, Tuple1<Long>>> mappedVertices = graph.mapVertices(new ToTuple1Mapper()).getVertices();
-        List<Vertex<Long, Tuple1<Long>>> result= mappedVertices.collect();
-
-		expectedResult = "1,(1)\n" +
-			"2,(2)\n" +
-			"3,(3)\n" +
-			"4,(4)\n" +
-			"5,(5)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithCustomType() throws Exception {
-		/*
-		 * Test mapVertices() and change the value type to a custom type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Vertex<Long, DummyCustomType>> mappedVertices = graph.mapVertices(new ToCustomTypeMapper()).getVertices();
-        List<Vertex<Long, DummyCustomType>> result= mappedVertices.collect();
-
-		expectedResult = "1,(T,1)\n" +
-			"2,(T,2)\n" +
-			"3,(T,3)\n" +
-			"4,(T,4)\n" +
-			"5,(T,5)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testWithCustomParametrizedType() throws Exception {
-		/*
-		 * Test mapVertices() and change the value type to a parameterized custom type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-		
-		DataSet<Vertex<Long, DummyCustomParameterizedType<Double>>> mappedVertices = graph.mapVertices(
-				new ToCustomParametrizedTypeMapper()).getVertices();
-        List<Vertex<Long, DummyCustomParameterizedType<Double>>> result= mappedVertices.collect();
-	
-		expectedResult = "1,(1.0,1)\n" +
-			"2,(2.0,2)\n" +
-			"3,(3.0,3)\n" +
-			"4,(4.0,4)\n" +
-			"5,(5.0,5)\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AddOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
-		public Long map(Vertex<Long, Long> value) throws Exception {
-			return value.getValue()+1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToStringMapper implements MapFunction<Vertex<Long, Long>, String> {
-		public String map(Vertex<Long, Long> vertex) throws Exception {
-			String stringValue;
-			if (vertex.getValue() == 1) {
-				stringValue = "one";
-			}
-			else if (vertex.getValue() == 2) {
-				stringValue = "two";
-			}
-			else if (vertex.getValue() == 3) {
-				stringValue = "three";
-			}
-			else if (vertex.getValue() == 4) {
-				stringValue = "four";
-			}
-			else if (vertex.getValue() == 5) {
-				stringValue = "five";
-			}
-			else {
-				stringValue = "";
-			}
-			return stringValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToTuple1Mapper implements MapFunction<Vertex<Long, Long>, Tuple1<Long>> {
-		public Tuple1<Long> map(Vertex<Long, Long> vertex) throws Exception {
-			Tuple1<Long> tupleValue = new Tuple1<Long>();
-			tupleValue.setFields(vertex.getValue());
-			return tupleValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToCustomTypeMapper implements MapFunction<Vertex<Long, Long>, DummyCustomType> {
-		public DummyCustomType map(Vertex<Long, Long> vertex) throws Exception {
-			DummyCustomType dummyValue = new DummyCustomType();
-			dummyValue.setIntField(vertex.getValue().intValue());						
-			return dummyValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ToCustomParametrizedTypeMapper implements MapFunction<Vertex<Long, Long>, 
-		DummyCustomParameterizedType<Double>> {
-		
-		public DummyCustomParameterizedType<Double> map(Vertex<Long, Long> vertex) throws Exception {
-			DummyCustomParameterizedType<Double> dummyValue = new DummyCustomParameterizedType<Double>();
-			dummyValue.setIntField(vertex.getValue().intValue());
-			dummyValue.setTField(new Double(vertex.getValue()));						
-			return dummyValue;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
deleted file mode 100644
index 3bb19fa..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
+++ /dev/null
@@ -1,615 +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.operations;
-
-import java.util.List;
-
-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.EdgeDirection;
-import org.apache.flink.graph.EdgesFunction;
-import org.apache.flink.graph.EdgesFunctionWithVertexValue;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.ReduceEdgesFunction;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class ReduceOnEdgesMethodsITCase extends MultipleProgramsTestBase {
-
-	public ReduceOnEdgesMethodsITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-    private String expectedResult;
-
-	@Test
-	public void testLowestWeightOutNeighbor() throws Exception {
-		/*
-		 * Get the lowest-weight out-neighbor
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
-				graph.groupReduceOnEdges(new SelectMinWeightNeighbor(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
-
-	
-		expectedResult = "1,2\n" +
-				"2,3\n" +
-				"3,4\n" +
-				"4,5\n" +
-				"5,1\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testLowestWeightInNeighbor() throws Exception {
-		/*
-		 * Get the lowest-weight in-neighbor
-		 * for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
-				graph.groupReduceOnEdges(new SelectMinWeightInNeighbor(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
-
-		expectedResult = "1,5\n" +
-					"2,1\n" + 
-					"3,1\n" +
-					"4,3\n" + 
-					"5,3\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllOutNeighbors() throws Exception {
-		/*
-		 * Get the all the out-neighbors for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
-				graph.groupReduceOnEdges(new SelectOutNeighbors(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
-
-		expectedResult = "1,2\n" +
-				"1,3\n" +
-				"2,3\n" +
-				"3,4\n" +
-				"3,5\n" +
-				"4,5\n" +
-				"5,1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllOutNeighborsNoValue() throws Exception {
-		/*
-		 * Get the all the out-neighbors for each vertex except for the vertex with id 5.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
-				graph.groupReduceOnEdges(new SelectOutNeighborsExcludeFive(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
-
-		expectedResult = "1,2\n" +
-				"1,3\n" +
-				"2,3\n" +
-				"3,4\n" +
-				"3,5\n" +
-				"4,5";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllOutNeighborsWithValueGreaterThanTwo() throws Exception {
-		/*
-		 * Get the all the out-neighbors for each vertex that have a value greater than two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
-				graph.groupReduceOnEdges(new SelectOutNeighborsValueGreaterThanTwo(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
-
-		expectedResult = "3,4\n" +
-				"3,5\n" +
-				"4,5\n" +
-				"5,1";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllInNeighbors() throws Exception {
-		/*
-		 * Get the all the in-neighbors for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
-				graph.groupReduceOnEdges(new SelectInNeighbors(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
-
-		expectedResult = "1,5\n" +
-				"2,1\n" +
-				"3,1\n" +
-				"3,2\n" +
-				"4,3\n" +
-				"5,3\n" +
-				"5,4";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllInNeighborsNoValue() throws Exception {
-		/*
-		 * Get the all the in-neighbors for each vertex except for the vertex with id 5.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
-				graph.groupReduceOnEdges(new SelectInNeighborsExceptFive(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
-
-		expectedResult = "1,5\n" +
-				"2,1\n" +
-				"3,1\n" +
-				"3,2\n" +
-				"4,3";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllInNeighborsWithValueGreaterThanTwo() throws Exception {
-		/*
-		 * Get the all the in-neighbors for each vertex that have a value greater than two.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
-				graph.groupReduceOnEdges(new SelectInNeighborsValueGreaterThanTwo(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
-
-		expectedResult = "3,1\n" +
-				"3,2\n" +
-				"4,3\n" +
-				"5,3\n" +
-				"5,4";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllNeighbors() throws Exception {
-		/*
-		 * Get the all the neighbors for each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
-				graph.groupReduceOnEdges(new SelectNeighbors(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
-
-		expectedResult = "1,2\n" +
-				"1,3\n" +
-				"1,5\n" +
-				"2,1\n" +
-				"2,3\n" +
-				"3,1\n" +
-				"3,2\n" +
-				"3,4\n" +
-				"3,5\n" +
-				"4,3\n" +
-				"4,5\n" +
-				"5,1\n" +
-				"5,3\n" +
-				"5,4";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllNeighborsNoValue() throws Exception {
-		/*
-		 * Get the all the neighbors for each vertex except for vertices with id 5 and 2.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
-				graph.groupReduceOnEdges(new SelectNeighborsExceptFiveAndTwo(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
-
-		expectedResult = "1,2\n" +
-				"1,3\n" +
-				"1,5\n" +
-				"3,1\n" +
-				"3,2\n" +
-				"3,4\n" +
-				"3,5\n" +
-				"4,3\n" +
-				"4,5";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testAllNeighborsWithValueGreaterThanFour() throws Exception {
-		/*
-		 * Get the all the neighbors for each vertex that have a value greater than four.
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
-				graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
-
-		expectedResult = "5,1\n" +
-				"5,3\n" +
-				"5,4";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testMaxWeightEdge() throws Exception {
-		/*
-		 * Get the maximum weight among all edges
-		 * of a vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithMaxEdgeWeight = 
-				graph.groupReduceOnEdges(new SelectMaxWeightNeighbor(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithMaxEdgeWeight.collect();
-
-		expectedResult = "1,51\n" +
-				"2,23\n" + 
-				"3,35\n" +
-				"4,45\n" + 
-				"5,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testLowestWeightOutNeighborNoValue() throws Exception {
-		/*
-		 * Get the lowest-weight out of all the out-neighbors
-		 * of each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
-				graph.reduceOnEdges(new SelectMinWeightNeighborNoValue(), EdgeDirection.OUT);
-		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
-
-		expectedResult = "1,12\n" +
-				"2,23\n" +
-				"3,34\n" +
-				"4,45\n" +
-				"5,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testLowestWeightInNeighborNoValue() throws Exception {
-		/*
-		 * Get the lowest-weight out of all the in-neighbors
-		 * of each vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
-				graph.reduceOnEdges(new SelectMinWeightNeighborNoValue(), EdgeDirection.IN);
-		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
-
-		expectedResult = "1,51\n" +
-				"2,12\n" +
-				"3,13\n" +
-				"4,34\n" +
-				"5,35\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@Test
-	public void testMaxWeightAllNeighbors() throws Exception {
-		/*
-		 * Get the maximum weight among all edges
-		 * of a vertex
-         */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
-				TestGraphUtils.getLongLongEdgeData(env), env);
-
-		DataSet<Tuple2<Long, Long>> verticesWithMaxEdgeWeight = 
-				graph.reduceOnEdges(new SelectMaxWeightNeighborNoValue(), EdgeDirection.ALL);
-		List<Tuple2<Long,Long>> result = verticesWithMaxEdgeWeight.collect();
-
-		expectedResult = "1,51\n" +
-				"2,23\n" + 
-				"3,35\n" +
-				"4,45\n" + 
-				"5,51\n";
-		
-		compareResultAsTuples(result, expectedResult);
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinWeightNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v,
-				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
-			
-			long weight = Long.MAX_VALUE;
-			long minNeighborId = 0;
-
-			for (Edge<Long, Long> edge: edges) {
-				if (edge.getValue() < weight) {
-					weight = edge.getValue();
-					minNeighborId = edge.getTarget();
-				}
-			}
-			out.collect(new Tuple2<Long, Long>(v.getId(), minNeighborId));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectMaxWeightNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v,
-				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
-			
-			long weight = Long.MIN_VALUE;
-
-			for (Edge<Long, Long> edge: edges) {
-				if (edge.getValue() > weight) {
-					weight = edge.getValue();
-				}
-			}
-			out.collect(new Tuple2<Long, Long>(v.getId(), weight));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinWeightNeighborNoValue implements ReduceEdgesFunction<Long> {
-
-		@Override
-		public Long reduceEdges(Long firstEdgeValue, Long secondEdgeValue) {
-			return Math.min(firstEdgeValue, secondEdgeValue);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectMaxWeightNeighborNoValue implements ReduceEdgesFunction<Long> {
-
-		@Override
-		public Long reduceEdges(Long firstEdgeValue, Long secondEdgeValue) {
-			return Math.max(firstEdgeValue, secondEdgeValue);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinWeightInNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v,
-				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
-			
-			long weight = Long.MAX_VALUE;
-			long minNeighborId = 0;
-			
-			for (Edge<Long, Long> edge: edges) {
-				if (edge.getValue() < weight) {
-					weight = edge.getValue();
-					minNeighborId = edge.getSource();
-				}
-			}
-			out.collect(new Tuple2<Long, Long>(v.getId(), minNeighborId));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectOutNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectOutNeighborsExcludeFive implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				if(edge.f0 != 5) {
-					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectOutNeighborsValueGreaterThanTwo implements
-			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for (Edge<Long, Long> edge: edges) {
-				if(v.getValue() > 2) {
-					out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectInNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectInNeighborsExceptFive implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				if(edge.f0 != 5) {
-					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectInNeighborsValueGreaterThanTwo implements
-			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for (Edge<Long, Long> edge: edges) {
-				if(v.getValue() > 2) {
-					out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for (Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				if (edge.f0 == edge.f1.getTarget()) {
-					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
-				} else {
-					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectNeighborsExceptFiveAndTwo implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for (Tuple2<Long, Edge<Long, Long>> edge : edges) {
-				if(edge.f0 != 5 && edge.f0 != 2) {
-					if (edge.f0 == edge.f1.getTarget()) {
-						out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
-					} else {
-						out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
-					}
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SelectNeighborsValueGreaterThanFour implements
-			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for(Edge<Long, Long> edge : edges) {
-				if(v.getValue() > 4) {
-					if(v.getId().equals(edge.getTarget())) {
-						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
-					} else {
-						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
-					}
-				}
-			}
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
deleted file mode 100644
index ab10947..0000000
--- a/flink-staging/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ /dev/null
@@ -1,142 +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.operations;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.EdgesFunctionWithVertexValue;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.apache.flink.util.Collector;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-public class ReduceOnEdgesWithExceptionITCase {
-
-	private static final int PARALLELISM = 4;
-
-	private static ForkableFlinkMiniCluster cluster;
-
-
-	@BeforeClass
-	public static void setupCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Error starting test cluster: " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void tearDownCluster() {
-		try {
-			cluster.stop();
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Cluster shutdown caused an exception: " + t.getMessage());
-		}
-	}
-
-	/**
-	 * Test groupReduceOnEdges() with an edge having a srcId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnEdgesInvalidEdgeSrcId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
-					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
-
-			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-	/**
-	 * Test groupReduceOnEdges() with an edge having a trgId that does not exist in the vertex DataSet
-	 */
-	@Test
-	public void testGroupReduceOnEdgesInvalidEdgeTrgId() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
-				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
-
-		try {
-			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
-					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
-
-			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
-			env.execute();
-		} catch (Exception e) {
-			// We expect the job to fail with an exception
-		}
-	}
-
-
-	@SuppressWarnings("serial")
-	private static final class SelectNeighborsValueGreaterThanFour implements
-			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
-
-		@Override
-		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
-								 Collector<Tuple2<Long, Long>> out) throws Exception {
-			for(Edge<Long, Long> edge : edges) {
-				if(v.getValue() > 4) {
-					if(v.getId().equals(edge.getTarget())) {
-						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
-					} else {
-						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
-					}
-				}
-			}
-		}
-	}
-}


[10/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
deleted file mode 100755
index b24f749..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java
+++ /dev/null
@@ -1,1948 +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;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-import java.util.List;
-import java.util.Arrays;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-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.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.graph.gsa.ApplyFunction;
-import org.apache.flink.graph.gsa.GSAConfiguration;
-import org.apache.flink.graph.gsa.GatherFunction;
-import org.apache.flink.graph.gsa.GatherSumApplyIteration;
-import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexCentricConfiguration;
-import org.apache.flink.graph.spargel.VertexCentricIteration;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.graph.utils.EdgeToTuple3Map;
-import org.apache.flink.graph.utils.Tuple2ToVertexMap;
-import org.apache.flink.graph.utils.Tuple3ToEdgeMap;
-import org.apache.flink.graph.utils.VertexToTuple2Map;
-import org.apache.flink.graph.validation.GraphValidator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.types.NullValue;
-
-/**
- * Represents a Graph consisting of {@link Edge edges} and {@link Vertex
- * vertices}.
- * 
- * 
- * @see org.apache.flink.graph.Edge
- * @see org.apache.flink.graph.Vertex
- * 
- * @param <K> the key type for edge and vertex identifiers
- * @param <VV> the value type for vertices
- * @param <EV> the value type for edges
- */
-@SuppressWarnings("serial")
-public class Graph<K, VV, EV> {
-
-	private final ExecutionEnvironment context;
-	private final DataSet<Vertex<K, VV>> vertices;
-	private final DataSet<Edge<K, EV>> edges;
-
-	/**
-	 * Creates a graph from two DataSets: vertices and edges
-	 * 
-	 * @param vertices a DataSet of vertices.
-	 * @param edges a DataSet of edges.
-	 * @param context the flink execution environment.
-	 */
-	private Graph(DataSet<Vertex<K, VV>> vertices, DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
-		this.vertices = vertices;
-		this.edges = edges;
-		this.context = context;
-	}
-
-	/**
-	 * Creates a graph from a Collection of vertices and a Collection of edges.
-	 * 
-	 * @param vertices a Collection of vertices.
-	 * @param edges a Collection of edges.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Vertex<K, VV>> vertices,
-			Collection<Edge<K, EV>> edges, ExecutionEnvironment context) {
-
-		return fromDataSet(context.fromCollection(vertices),
-				context.fromCollection(edges), context);
-	}
-
-	/**
-	 * Creates a graph from a Collection of edges, vertices are induced from the
-	 * edges. Vertices are created automatically and their values are set to
-	 * NullValue.
-	 * 
-	 * @param edges a Collection of vertices.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, EV> Graph<K, NullValue, EV> fromCollection(Collection<Edge<K, EV>> edges,
-			ExecutionEnvironment context) {
-
-		return fromDataSet(context.fromCollection(edges), context);
-	}
-
-	/**
-	 * Creates a graph from a Collection of edges, vertices are induced from the
-	 * edges and vertex values are calculated by a mapper function. Vertices are
-	 * created automatically and their values are set by applying the provided
-	 * map function to the vertex ids.
-	 * 
-	 * @param edges a Collection of edges.
-	 * @param mapper the mapper function.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromCollection(Collection<Edge<K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-
-		return fromDataSet(context.fromCollection(edges), mapper, context);
-	}
-
-	/**
-	 * Creates a graph from a DataSet of vertices and a DataSet of edges.
-	 * 
-	 * @param vertices a DataSet of vertices.
-	 * @param edges a DataSet of edges.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromDataSet(DataSet<Vertex<K, VV>> vertices,
-			DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
-
-		return new Graph<K, VV, EV>(vertices, edges, context);
-	}
-
-	/**
-	 * Creates a graph from a DataSet of edges, vertices are induced from the
-	 * edges. Vertices are created automatically and their values are set to
-	 * NullValue.
-	 * 
-	 * @param edges a DataSet of edges.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, EV> Graph<K, NullValue, EV> fromDataSet(
-			DataSet<Edge<K, EV>> edges, ExecutionEnvironment context) {
-
-		DataSet<Vertex<K, NullValue>> vertices = edges.flatMap(new EmitSrcAndTarget<K, EV>()).distinct();
-
-		return new Graph<K, NullValue, EV>(vertices, edges, context);
-	}
-
-	private static final class EmitSrcAndTarget<K, EV> implements FlatMapFunction<
-		Edge<K, EV>, Vertex<K, NullValue>> {
-
-		public void flatMap(Edge<K, EV> edge, Collector<Vertex<K, NullValue>> out) {
-			out.collect(new Vertex<K, NullValue>(edge.f0, NullValue.getInstance()));
-			out.collect(new Vertex<K, NullValue>(edge.f1, NullValue.getInstance()));
-		}
-	}
-
-	/**
-	 * Creates a graph from a DataSet of edges, vertices are induced from the
-	 * edges and vertex values are calculated by a mapper function. Vertices are
-	 * created automatically and their values are set by applying the provided
-	 * map function to the vertex ids.
-	 * 
-	 * @param edges a DataSet of edges.
-	 * @param mapper the mapper function.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromDataSet(DataSet<Edge<K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-
-		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
-
-		TypeInformation<VV> valueType = TypeExtractor.createTypeInfo(
-				MapFunction.class, mapper.getClass(), 1, null, null);
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		TypeInformation<Vertex<K, VV>> returnType = (TypeInformation<Vertex<K, VV>>) new TupleTypeInfo(
-				Vertex.class, keyType, valueType);
-
-		DataSet<Vertex<K, VV>> vertices = edges
-				.flatMap(new EmitSrcAndTargetAsTuple1<K, EV>()).distinct()
-				.map(new MapFunction<Tuple1<K>, Vertex<K, VV>>() {
-					public Vertex<K, VV> map(Tuple1<K> value) throws Exception {
-						return new Vertex<K, VV>(value.f0, mapper.map(value.f0));
-					}
-				}).returns(returnType).withForwardedFields("f0");
-
-		return new Graph<K, VV, EV>(vertices, edges, context);
-	}
-
-	private static final class EmitSrcAndTargetAsTuple1<K, EV> implements FlatMapFunction<
-		Edge<K, EV>, Tuple1<K>> {
-
-		public void flatMap(Edge<K, EV> edge, Collector<Tuple1<K>> out) {
-			out.collect(new Tuple1<K>(edge.f0));
-			out.collect(new Tuple1<K>(edge.f1));
-		}
-	}
-
-	/**
-	 * Creates a graph from a DataSet of Tuple objects for vertices and edges.
-	 * 
-	 * Vertices with value are created from Tuple2, Edges with value are created
-	 * from Tuple3.
-	 * 
-	 * @param vertices a DataSet of Tuple2.
-	 * @param edges a DataSet of Tuple3.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromTupleDataSet(DataSet<Tuple2<K, VV>> vertices,
-			DataSet<Tuple3<K, K, EV>> edges, ExecutionEnvironment context) {
-
-		DataSet<Vertex<K, VV>> vertexDataSet = vertices.map(new Tuple2ToVertexMap<K, VV>());
-		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
-		return fromDataSet(vertexDataSet, edgeDataSet, context);
-	}
-
-	/**
-	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
-	 * induced from the edges.
-	 * 
-	 * Edges with value are created from Tuple3. Vertices are created
-	 * automatically and their values are set to NullValue.
-	 * 
-	 * @param edges a DataSet of Tuple3.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, EV> Graph<K, NullValue, EV> fromTupleDataSet(DataSet<Tuple3<K, K, EV>> edges,
-			ExecutionEnvironment context) {
-
-		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
-		return fromDataSet(edgeDataSet, context);
-	}
-
-	/**
-	 * Creates a graph from a DataSet of Tuple objects for edges, vertices are
-	 * induced from the edges and vertex values are calculated by a mapper
-	 * function. Edges with value are created from Tuple3. Vertices are created
-	 * automatically and their values are set by applying the provided map
-	 * function to the vertex ids.
-	 * 
-	 * @param edges a DataSet of Tuple3.
-	 * @param mapper the mapper function.
-	 * @param context the flink execution environment.
-	 * @return the newly created graph.
-	 */
-	public static <K, VV, EV> Graph<K, VV, EV> fromTupleDataSet(DataSet<Tuple3<K, K, EV>> edges,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-
-		DataSet<Edge<K, EV>> edgeDataSet = edges.map(new Tuple3ToEdgeMap<K, EV>());
-		return fromDataSet(edgeDataSet, mapper, context);
-	}
-
-	/**
-	* Creates a Graph from a CSV file of vertices and a CSV file of edges.
-	* 
-	* @param verticesPath path to a CSV file with the Vertex data.
-	* @param edgesPath path to a CSV file with the Edge data
-	* @param context the Flink execution environment.
-	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader}, 
-	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
-	* 
-	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
-	*/
-	public static GraphCsvReader fromCsvReader(String verticesPath, String edgesPath, ExecutionEnvironment context) {
-		return new GraphCsvReader(verticesPath, edgesPath, context);
-	}
-
-	/** 
-	* Creates a graph from a CSV file of edges. Vertices will be created automatically.
-	*
-	* @param edgesPath a path to a CSV file with the Edges data
-	* @param context the execution environment.
-	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
-	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
-	* 
-	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
-	*/
-	public static GraphCsvReader fromCsvReader(String edgesPath, ExecutionEnvironment context) {
-		return new GraphCsvReader(edgesPath, context);
-	}
-
-	/** 
-	 * Creates a graph from a CSV file of edges. Vertices will be created automatically and
-	 * Vertex values are set by the provided mapper.
-	 *
-	 * @param edgesPath a path to a CSV file with the Edge data
-	 * @param mapper the mapper function.
-	 * @param context the execution environment.
-	 * @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
-	 * on which calling methods to specify types of the Vertex ID, Vertex Value and Edge value returns a Graph.
-	 * 
-	 * @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	 * {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	 * {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	 * {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
-	 */
-	public static <K, VV> GraphCsvReader fromCsvReader(String edgesPath,
-			final MapFunction<K, VV> mapper, ExecutionEnvironment context) {
-		return new GraphCsvReader(edgesPath, mapper, context);
-	}
-
-	/**
-	 * @return the flink execution environment.
-	 */
-	public ExecutionEnvironment getContext() {
-		return this.context;
-	}
-
-	/**
-	 * Function that checks whether a Graph is a valid Graph,
-	 * as defined by the given {@link GraphValidator}.
-	 * 
-	 * @return true if the Graph is valid.
-	 */
-	public Boolean validate(GraphValidator<K, VV, EV> validator) throws Exception {
-		return validator.validate(this);
-	}
-
-	/**
-	 * @return the vertex DataSet.
-	 */
-	public DataSet<Vertex<K, VV>> getVertices() {
-		return vertices;
-	}
-
-	/**
-	 * @return the edge DataSet.
-	 */
-	public DataSet<Edge<K, EV>> getEdges() {
-		return edges;
-	}
-
-	/**
-	 * @return the vertex DataSet as Tuple2.
-	 */
-	public DataSet<Tuple2<K, VV>> getVerticesAsTuple2() {
-		return vertices.map(new VertexToTuple2Map<K, VV>());
-	}
-
-	/**
-	 * @return the edge DataSet as Tuple3.
-	 */
-	public DataSet<Tuple3<K, K, EV>> getEdgesAsTuple3() {
-		return edges.map(new EdgeToTuple3Map<K, EV>());
-	}
-
-	/**
-	 * This method allows access to the graph's edge values along with its source and target vertex values.
-	 *
-	 * @return a triplet DataSet consisting of (srcVertexId, trgVertexId, srcVertexValue, trgVertexValue, edgeValue)
-	 */
-	public DataSet<Triplet<K, VV, EV>> getTriplets() {
-		return this.getVertices().join(this.getEdges()).where(0).equalTo(0)
-				.with(new ProjectEdgeWithSrcValue<K, VV, EV>())
-				.join(this.getVertices()).where(1).equalTo(0)
-				.with(new ProjectEdgeWithVertexValues<K, VV, EV>());
-	}
-
-	@ForwardedFieldsFirst("f1->f2")
-	@ForwardedFieldsSecond("f0; f1; f2->f3")
-	private static final class ProjectEdgeWithSrcValue<K, VV, EV> implements
-			FlatJoinFunction<Vertex<K, VV>, Edge<K, EV>, Tuple4<K, K, VV, EV>> {
-
-		@Override
-		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple4<K, K, VV, EV>> collector)
-				throws Exception {
-
-			collector.collect(new Tuple4<K, K, VV, EV>(edge.getSource(), edge.getTarget(), vertex.getValue(),
-					edge.getValue()));
-		}
-	}
-
-	@ForwardedFieldsFirst("f0; f1; f2; f3->f4")
-	@ForwardedFieldsSecond("f1->f3")
-	private static final class ProjectEdgeWithVertexValues<K, VV, EV> implements
-			FlatJoinFunction<Tuple4<K, K, VV, EV>, Vertex<K, VV>, Triplet<K, VV, EV>> {
-
-		@Override
-		public void join(Tuple4<K, K, VV, EV> tripletWithSrcValSet,
-						Vertex<K, VV> vertex, Collector<Triplet<K, VV, EV>> collector) throws Exception {
-
-			collector.collect(new Triplet<K, VV, EV>(tripletWithSrcValSet.f0, tripletWithSrcValSet.f1,
-					tripletWithSrcValSet.f2, vertex.getValue(), tripletWithSrcValSet.f3));
-		}
-	}
-
-	/**
-	 * Apply a function to the attribute of each vertex in the graph.
-	 * 
-	 * @param mapper the map function to apply.
-	 * @return a new graph
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper) {
-
-		TypeInformation<K> keyType = ((TupleTypeInfo<?>) vertices.getType()).getTypeAt(0);
-
-		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null);
-
-		TypeInformation<Vertex<K, NV>> returnType = (TypeInformation<Vertex<K, NV>>) new TupleTypeInfo(
-				Vertex.class, keyType, valueType);
-
-		return mapVertices(mapper, returnType);
-	}
-
-	/**
-	 * Apply a function to the attribute of each vertex in the graph.
-	 *
-	 * @param mapper the map function to apply.
-	 * @param returnType the explicit return type.
-	 * @return a new graph
-	 */
-	public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper, TypeInformation<Vertex<K,NV>> returnType) {
-		DataSet<Vertex<K, NV>> mappedVertices = vertices.map(
-				new MapFunction<Vertex<K, VV>, Vertex<K, NV>>() {
-					public Vertex<K, NV> map(Vertex<K, VV> value) throws Exception {
-						return new Vertex<K, NV>(value.f0, mapper.map(value));
-					}
-				})
-				.returns(returnType)
-				.withForwardedFields("f0");
-
-		return new Graph<K, NV, EV>(mappedVertices, this.edges, this.context);
-	}
-
-	/**
-	 * Apply a function to the attribute of each edge in the graph.
-	 * 
-	 * @param mapper the map function to apply.
-	 * @return a new graph
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper) {
-
-		TypeInformation<K> keyType = ((TupleTypeInfo<?>) edges.getType()).getTypeAt(0);
-
-		TypeInformation<NV> valueType = TypeExtractor.createTypeInfo(MapFunction.class, mapper.getClass(), 1, null, null);
-
-		TypeInformation<Edge<K, NV>> returnType = (TypeInformation<Edge<K, NV>>) new TupleTypeInfo(
-				Edge.class, keyType, keyType, valueType);
-
-		return mapEdges(mapper, returnType);
-	}
-
-	/**
-	 * Apply a function to the attribute of each edge in the graph.
-	 *
-	 * @param mapper the map function to apply.
-	 * @param returnType the explicit return type.
-	 * @return a new graph
-	 */
-	public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper, TypeInformation<Edge<K,NV>> returnType) {
-		DataSet<Edge<K, NV>> mappedEdges = edges.map(
-				new MapFunction<Edge<K, EV>, Edge<K, NV>>() {
-					public Edge<K, NV> map(Edge<K, EV> value) throws Exception {
-						return new Edge<K, NV>(value.f0, value.f1, mapper
-								.map(value));
-					}
-				})
-				.returns(returnType)
-				.withForwardedFields("f0; f1");
-
-		return new Graph<K, VV, NV>(this.vertices, mappedEdges, this.context);
-	}
-
-	/**
-	 * Joins the vertex DataSet of this graph with an input DataSet and applies
-	 * a UDF on the resulted values.
-	 * 
-	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @return a new graph where the vertex values have been updated.
-	 */
-	public <T> Graph<K, VV, EV> joinWithVertices(DataSet<Tuple2<K, T>> inputDataSet, 
-			final MapFunction<Tuple2<VV, T>, VV> mapper) {
-
-		DataSet<Vertex<K, VV>> resultedVertices = this.getVertices()
-				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToVertexValues<K, VV, T>(mapper));
-		return new Graph<K, VV, EV>(resultedVertices, this.edges, this.context);
-	}
-
-	private static final class ApplyCoGroupToVertexValues<K, VV, T>
-			implements CoGroupFunction<Vertex<K, VV>, Tuple2<K, T>, Vertex<K, VV>> {
-
-		private MapFunction<Tuple2<VV, T>, VV> mapper;
-
-		public ApplyCoGroupToVertexValues(MapFunction<Tuple2<VV, T>, VV> mapper) {
-			this.mapper = mapper;
-		}
-
-		@Override
-		public void coGroup(Iterable<Vertex<K, VV>> vertices,
-				Iterable<Tuple2<K, T>> input, Collector<Vertex<K, VV>> collector) throws Exception {
-
-			final Iterator<Vertex<K, VV>> vertexIterator = vertices.iterator();
-			final Iterator<Tuple2<K, T>> inputIterator = input.iterator();
-
-			if (vertexIterator.hasNext()) {
-				if (inputIterator.hasNext()) {
-					final Tuple2<K, T> inputNext = inputIterator.next();
-
-					collector.collect(new Vertex<K, VV>(inputNext.f0, mapper
-							.map(new Tuple2<VV, T>(vertexIterator.next().f1,
-									inputNext.f1))));
-				} else {
-					collector.collect(vertexIterator.next());
-				}
-
-			}
-		}
-	}
-
-	/**
-	 * Joins the edge DataSet with an input DataSet on a composite key of both
-	 * source and target and applies a UDF on the resulted values.
-	 * 
-	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
-	public <T> Graph<K, VV, EV> joinWithEdges(DataSet<Tuple3<K, K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
-
-		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
-				.coGroup(inputDataSet).where(0, 1).equalTo(0, 1)
-				.with(new ApplyCoGroupToEdgeValues<K, EV, T>(mapper));
-		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
-	}
-
-	private static final class ApplyCoGroupToEdgeValues<K, EV, T>
-			implements CoGroupFunction<Edge<K, EV>, Tuple3<K, K, T>, Edge<K, EV>> {
-
-		private MapFunction<Tuple2<EV, T>, EV> mapper;
-
-		public ApplyCoGroupToEdgeValues(MapFunction<Tuple2<EV, T>, EV> mapper) {
-			this.mapper = mapper;
-		}
-
-		@Override
-		public void coGroup(Iterable<Edge<K, EV>> edges, Iterable<Tuple3<K, K, T>> input,
-				Collector<Edge<K, EV>> collector) throws Exception {
-
-			final Iterator<Edge<K, EV>> edgesIterator = edges.iterator();
-			final Iterator<Tuple3<K, K, T>> inputIterator = input.iterator();
-
-			if (edgesIterator.hasNext()) {
-				if (inputIterator.hasNext()) {
-					final Tuple3<K, K, T> inputNext = inputIterator.next();
-
-					collector.collect(new Edge<K, EV>(inputNext.f0,
-							inputNext.f1, mapper.map(new Tuple2<EV, T>(
-									edgesIterator.next().f2, inputNext.f2))));
-				} else {
-					collector.collect(edgesIterator.next());
-				}
-			}
-		}
-	}
-
-	/**
-	 * Joins the edge DataSet with an input DataSet on the source key of the
-	 * edges and the first attribute of the input DataSet and applies a UDF on
-	 * the resulted values. In case the inputDataSet contains the same key more
-	 * than once, only the first value will be considered.
-	 * 
-	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
-	public <T> Graph<K, VV, EV> joinWithEdgesOnSource(DataSet<Tuple2<K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
-
-		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
-				.coGroup(inputDataSet).where(0).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
-
-		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
-	}
-
-	private static final class ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>
-			implements CoGroupFunction<Edge<K, EV>, Tuple2<K, T>, Edge<K, EV>> {
-
-		private MapFunction<Tuple2<EV, T>, EV> mapper;
-
-		public ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget(
-				MapFunction<Tuple2<EV, T>, EV> mapper) {
-			this.mapper = mapper;
-		}
-
-		@Override
-		public void coGroup(Iterable<Edge<K, EV>> edges,
-				Iterable<Tuple2<K, T>> input, Collector<Edge<K, EV>> collector) throws Exception {
-
-			final Iterator<Edge<K, EV>> edgesIterator = edges.iterator();
-			final Iterator<Tuple2<K, T>> inputIterator = input.iterator();
-
-			if (inputIterator.hasNext()) {
-				final Tuple2<K, T> inputNext = inputIterator.next();
-
-				while (edgesIterator.hasNext()) {
-					Edge<K, EV> edgesNext = edgesIterator.next();
-
-					collector.collect(new Edge<K, EV>(edgesNext.f0,
-							edgesNext.f1, mapper.map(new Tuple2<EV, T>(
-									edgesNext.f2, inputNext.f1))));
-				}
-
-			} else {
-				while (edgesIterator.hasNext()) {
-					collector.collect(edgesIterator.next());
-				}
-			}
-		}
-	}
-
-	/**
-	 * Joins the edge DataSet with an input DataSet on the target key of the
-	 * edges and the first attribute of the input DataSet and applies a UDF on
-	 * the resulted values. Should the inputDataSet contain the same key more
-	 * than once, only the first value will be considered.
-	 * 
-	 * @param inputDataSet the DataSet to join with.
-	 * @param mapper the UDF map function to apply.
-	 * @param <T> the return type
-	 * @return a new graph where the edge values have been updated.
-	 */
-	public <T> Graph<K, VV, EV> joinWithEdgesOnTarget(DataSet<Tuple2<K, T>> inputDataSet,
-			final MapFunction<Tuple2<EV, T>, EV> mapper) {
-
-		DataSet<Edge<K, EV>> resultedEdges = this.getEdges()
-				.coGroup(inputDataSet).where(1).equalTo(0)
-				.with(new ApplyCoGroupToEdgeValuesOnEitherSourceOrTarget<K, EV, T>(mapper));
-
-		return new Graph<K, VV, EV>(this.vertices, resultedEdges, this.context);
-	}
-
-	/**
-	 * Apply filtering functions to the graph and return a sub-graph that
-	 * satisfies the predicates for both vertices and edges.
-	 * 
-	 * @param vertexFilter the filter function for vertices.
-	 * @param edgeFilter the filter function for edges.
-	 * @return the resulting sub-graph.
-	 */
-	public Graph<K, VV, EV> subgraph(FilterFunction<Vertex<K, VV>> vertexFilter, FilterFunction<Edge<K, EV>> edgeFilter) {
-
-		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
-
-		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
-				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
-				.join(filteredVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>());
-
-		DataSet<Edge<K, EV>> filteredEdges = remainingEdges.filter(edgeFilter);
-
-		return new Graph<K, VV, EV>(filteredVertices, filteredEdges,
-				this.context);
-	}
-
-	/**
-	 * Apply a filtering function to the graph and return a sub-graph that
-	 * satisfies the predicates only for the vertices.
-	 * 
-	 * @param vertexFilter the filter function for vertices.
-	 * @return the resulting sub-graph.
-	 */
-	public Graph<K, VV, EV> filterOnVertices(FilterFunction<Vertex<K, VV>> vertexFilter) {
-
-		DataSet<Vertex<K, VV>> filteredVertices = this.vertices.filter(vertexFilter);
-
-		DataSet<Edge<K, EV>> remainingEdges = this.edges.join(filteredVertices)
-				.where(0).equalTo(0).with(new ProjectEdge<K, VV, EV>())
-				.join(filteredVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>());
-
-		return new Graph<K, VV, EV>(filteredVertices, remainingEdges, this.context);
-	}
-
-	/**
-	 * Apply a filtering function to the graph and return a sub-graph that
-	 * satisfies the predicates only for the edges.
-	 * 
-	 * @param edgeFilter the filter function for edges.
-	 * @return the resulting sub-graph.
-	 */
-	public Graph<K, VV, EV> filterOnEdges(FilterFunction<Edge<K, EV>> edgeFilter) {
-		DataSet<Edge<K, EV>> filteredEdges = this.edges.filter(edgeFilter);
-
-		return new Graph<K, VV, EV>(this.vertices, filteredEdges, this.context);
-	}
-
-	@ForwardedFieldsFirst("f0; f1; f2")
-	private static final class ProjectEdge<K, VV, EV> implements FlatJoinFunction<
-		Edge<K, EV>, Vertex<K, VV>, Edge<K, EV>> {
-		public void join(Edge<K, EV> first, Vertex<K, VV> second, Collector<Edge<K, EV>> out) {
-			out.collect(first);
-		}
-	}
-
-	/**
-	 * Return the out-degree of all vertices in the graph
-	 * 
-	 * @return A DataSet of Tuple2<vertexId, outDegree>
-	 */
-	public DataSet<Tuple2<K, Long>> outDegrees() {
-
-		return vertices.coGroup(edges).where(0).equalTo(0).with(new CountNeighborsCoGroup<K, VV, EV>());
-	}
-
-	private static final class CountNeighborsCoGroup<K, VV, EV>
-			implements CoGroupFunction<Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Long>> {
-		@SuppressWarnings("unused")
-		public void coGroup(Iterable<Vertex<K, VV>> vertex,	Iterable<Edge<K, EV>> outEdges,
-				Collector<Tuple2<K, Long>> out) {
-			long count = 0;
-			for (Edge<K, EV> edge : outEdges) {
-				count++;
-			}
-
-			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
-
-			if(vertexIterator.hasNext()) {
-				out.collect(new Tuple2<K, Long>(vertexIterator.next().f0, count));
-			} else {
-				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
-			}
-		}
-	}
-
-	/**
-	 * Return the in-degree of all vertices in the graph
-	 * 
-	 * @return A DataSet of Tuple2<vertexId, inDegree>
-	 */
-	public DataSet<Tuple2<K, Long>> inDegrees() {
-
-		return vertices.coGroup(edges).where(0).equalTo(1).with(new CountNeighborsCoGroup<K, VV, EV>());
-	}
-
-	/**
-	 * Return the degree of all vertices in the graph
-	 * 
-	 * @return A DataSet of Tuple2<vertexId, degree>
-	 */
-	public DataSet<Tuple2<K, Long>> getDegrees() {
-		return outDegrees().union(inDegrees()).groupBy(0).sum(1);
-	}
-
-	/**
-	 * This operation adds all inverse-direction edges to the graph.
-	 * 
-	 * @return the undirected graph.
-	 */
-	public Graph<K, VV, EV> getUndirected() {
-
-		DataSet<Edge<K, EV>> undirectedEdges = edges.flatMap(new RegularAndReversedEdgesMap<K, EV>());
-		return new Graph<K, VV, EV>(vertices, undirectedEdges, this.context);
-	}
-
-	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges has access to the vertex value.
-	 * 
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
-											EdgeDirection direction) throws IllegalArgumentException {
-
-		switch (direction) {
-		case IN:
-			return vertices.coGroup(edges).where(0).equalTo(1)
-					.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction));
-		case OUT:
-			return vertices.coGroup(edges).where(0).equalTo(0)
-					.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction));
-		case ALL:
-			return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>()))
-					.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>(edgesFunction));
-		default:
-			throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges has access to the vertex value.
-	 *
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> edgesFunction,
-											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
-
-		switch (direction) {
-			case IN:
-				return vertices.coGroup(edges).where(0).equalTo(1)
-						.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
-			case OUT:
-				return vertices.coGroup(edges).where(0).equalTo(0)
-						.with(new ApplyCoGroupFunction<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
-			case ALL:
-				return vertices.coGroup(edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>()))
-						.where(0).equalTo(0).with(new ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>(edgesFunction)).returns(typeInfo);
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges only has access to the vertex id (not the vertex value).
-	 * 
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @return a dataset of T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
-											EdgeDirection direction) throws IllegalArgumentException {
-
-		switch (direction) {
-		case IN:
-			return edges.map(new ProjectVertexIdMap<K, EV>(1))
-					.withForwardedFields("f1->f0")
-					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
-		case OUT:
-			return edges.map(new ProjectVertexIdMap<K, EV>(0))
-					.withForwardedFields("f0")
-					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
-		case ALL:
-			return edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>())
-					.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction));
-		default:
-			throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the edges of each vertex. The function applied
-	 * on the edges only has access to the vertex id (not the vertex value).
-	 *
-	 * @param edgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @param <T>
-	 *            the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnEdges(EdgesFunction<K, EV, T> edgesFunction,
-											EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
-
-		switch (direction) {
-			case IN:
-				return edges.map(new ProjectVertexIdMap<K, EV>(1))
-						.withForwardedFields("f1->f0")
-						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
-			case OUT:
-				return edges.map(new ProjectVertexIdMap<K, EV>(0))
-						.withForwardedFields("f0")
-						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
-			case ALL:
-				return edges.flatMap(new EmitOneEdgePerNode<K, VV, EV>())
-						.groupBy(0).reduceGroup(new ApplyGroupReduceFunction<K, EV, T>(edgesFunction)).returns(typeInfo);
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	private static final class ProjectVertexIdMap<K, EV> implements MapFunction<
-		Edge<K, EV>, Tuple2<K, Edge<K, EV>>> {
-
-		private int fieldPosition;
-
-		public ProjectVertexIdMap(int position) {
-			this.fieldPosition = position;
-		}
-
-		@SuppressWarnings("unchecked")
-		public Tuple2<K, Edge<K, EV>> map(Edge<K, EV> edge) {
-			return new Tuple2<K, Edge<K, EV>>((K) edge.getField(fieldPosition),	edge);
-		}
-	}
-
-	private static final class ProjectVertexWithEdgeValueMap<K, EV>	implements MapFunction<
-		Edge<K, EV>, Tuple2<K, EV>> {
-
-		private int fieldPosition;
-
-		public ProjectVertexWithEdgeValueMap(int position) {
-			this.fieldPosition = position;
-		}
-
-		@SuppressWarnings("unchecked")
-		public Tuple2<K, EV> map(Edge<K, EV> edge) {
-			return new Tuple2<K, EV>((K) edge.getField(fieldPosition),	edge.getValue());
-		}
-	}
-
-	private static final class ApplyGroupReduceFunction<K, EV, T> implements GroupReduceFunction<
-		Tuple2<K, Edge<K, EV>>, T>,	ResultTypeQueryable<T> {
-
-		private EdgesFunction<K, EV, T> function;
-
-		public ApplyGroupReduceFunction(EdgesFunction<K, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void reduce(Iterable<Tuple2<K, Edge<K, EV>>> edges, Collector<T> out) throws Exception {
-			function.iterateEdges(edges, out);
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(EdgesFunction.class, function.getClass(), 2, null, null);
-		}
-	}
-
-	private static final class EmitOneEdgePerNode<K, VV, EV> implements FlatMapFunction<
-		Edge<K, EV>, Tuple2<K, Edge<K, EV>>> {
-
-		public void flatMap(Edge<K, EV> edge, Collector<Tuple2<K, Edge<K, EV>>> out) {
-			out.collect(new Tuple2<K, Edge<K, EV>>(edge.getSource(), edge));
-			out.collect(new Tuple2<K, Edge<K, EV>>(edge.getTarget(), edge));
-		}
-	}
-
-	private static final class EmitOneVertexWithEdgeValuePerNode<K, EV>	implements FlatMapFunction<
-		Edge<K, EV>, Tuple2<K, EV>> {
-
-		public void flatMap(Edge<K, EV> edge, Collector<Tuple2<K, EV>> out) {
-			out.collect(new Tuple2<K, EV>(edge.getSource(), edge.getValue()));
-			out.collect(new Tuple2<K, EV>(edge.getTarget(), edge.getValue()));
-		}
-	}
-
-	private static final class EmitOneEdgeWithNeighborPerNode<K, EV> implements FlatMapFunction<
-		Edge<K, EV>, Tuple3<K, K, Edge<K, EV>>> {
-
-		public void flatMap(Edge<K, EV> edge, Collector<Tuple3<K, K, Edge<K, EV>>> out) {
-			out.collect(new Tuple3<K, K, Edge<K, EV>>(edge.getSource(), edge.getTarget(), edge));
-			out.collect(new Tuple3<K, K, Edge<K, EV>>(edge.getTarget(), edge.getSource(), edge));
-		}
-	}
-
-	private static final class ApplyCoGroupFunction<K, VV, EV, T> implements CoGroupFunction<
-		Vertex<K, VV>, Edge<K, EV>, T>, ResultTypeQueryable<T> {
-
-		private EdgesFunctionWithVertexValue<K, VV, EV, T> function;
-
-		public ApplyCoGroupFunction(EdgesFunctionWithVertexValue<K, VV, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void coGroup(Iterable<Vertex<K, VV>> vertex,
-				Iterable<Edge<K, EV>> edges, Collector<T> out) throws Exception {
-
-			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
-
-			if(vertexIterator.hasNext()) {
-				function.iterateEdges(vertexIterator.next(), edges, out);
-			} else {
-				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
-			}
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3,
-					null, null);
-		}
-	}
-
-	private static final class ApplyCoGroupFunctionOnAllEdges<K, VV, EV, T>
-			implements	CoGroupFunction<Vertex<K, VV>, Tuple2<K, Edge<K, EV>>, T>, ResultTypeQueryable<T> {
-
-		private EdgesFunctionWithVertexValue<K, VV, EV, T> function;
-
-		public ApplyCoGroupFunctionOnAllEdges(EdgesFunctionWithVertexValue<K, VV, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void coGroup(Iterable<Vertex<K, VV>> vertex,	final Iterable<Tuple2<K, Edge<K, EV>>> keysWithEdges,
-				Collector<T> out) throws Exception {
-
-			final Iterator<Edge<K, EV>> edgesIterator = new Iterator<Edge<K, EV>>() {
-
-				final Iterator<Tuple2<K, Edge<K, EV>>> keysWithEdgesIterator = keysWithEdges.iterator();
-
-				@Override
-				public boolean hasNext() {
-					return keysWithEdgesIterator.hasNext();
-				}
-
-				@Override
-				public Edge<K, EV> next() {
-					return keysWithEdgesIterator.next().f1;
-				}
-
-				@Override
-				public void remove() {
-					keysWithEdgesIterator.remove();
-				}
-			};
-
-			Iterable<Edge<K, EV>> edgesIterable = new Iterable<Edge<K, EV>>() {
-				public Iterator<Edge<K, EV>> iterator() {
-					return edgesIterator;
-				}
-			};
-
-			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
-
-			if(vertexIterator.hasNext()) {
-				function.iterateEdges(vertexIterator.next(), edgesIterable, out);
-			} else {
-				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
-			}
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(EdgesFunctionWithVertexValue.class, function.getClass(), 3,
-					null, null);
-		}
-	}
-
-	@ForwardedFields("f0->f1; f1->f0; f2")
-	private static final class ReverseEdgesMap<K, EV>
-			implements MapFunction<Edge<K, EV>, Edge<K, EV>> {
-
-		public Edge<K, EV> map(Edge<K, EV> value) {
-			return new Edge<K, EV>(value.f1, value.f0, value.f2);
-		}
-	}
-
-	private static final class RegularAndReversedEdgesMap<K, EV>
-			implements FlatMapFunction<Edge<K, EV>, Edge<K, EV>> {
-
-		@Override
-		public void flatMap(Edge<K, EV> edge, Collector<Edge<K, EV>> out) throws Exception {
-			out.collect(new Edge<K, EV>(edge.f0, edge.f1, edge.f2));
-			out.collect(new Edge<K, EV>(edge.f1, edge.f0, edge.f2));
-		}
-	}
-
-	/**
-	 * Reverse the direction of the edges in the graph
-	 * 
-	 * @return a new graph with all edges reversed
-	 * @throws UnsupportedOperationException
-	 */
-	public Graph<K, VV, EV> reverse() throws UnsupportedOperationException {
-		DataSet<Edge<K, EV>> reversedEdges = edges.map(new ReverseEdgesMap<K, EV>());
-		return new Graph<K, VV, EV>(vertices, reversedEdges, this.context);
-	}
-
-	/**
-	 * @return a long integer representing the number of vertices
-	 */
-	public long numberOfVertices() throws Exception {
-		return vertices.count();
-	}
-
-	/**
-	 * @return a long integer representing the number of edges
-	 */
-	public long numberOfEdges() throws Exception {
-		return edges.count();
-	}
-
-	/**
-	 * @return The IDs of the vertices as DataSet
-	 */
-	public DataSet<K> getVertexIds() {
-		return vertices.map(new ExtractVertexIDMapper<K, VV>());
-	}
-
-	private static final class ExtractVertexIDMapper<K, VV>
-			implements MapFunction<Vertex<K, VV>, K> {
-		@Override
-		public K map(Vertex<K, VV> vertex) {
-			return vertex.f0;
-		}
-	}
-
-	/**
-	 * @return The IDs of the edges as DataSet
-	 */
-	public DataSet<Tuple2<K, K>> getEdgeIds() {
-		return edges.map(new ExtractEdgeIDsMapper<K, EV>());
-	}
-
-	@ForwardedFields("f0; f1")
-	private static final class ExtractEdgeIDsMapper<K, EV>
-			implements MapFunction<Edge<K, EV>, Tuple2<K, K>> {
-		@Override
-		public Tuple2<K, K> map(Edge<K, EV> edge) throws Exception {
-			return new Tuple2<K, K>(edge.f0, edge.f1);
-		}
-	}
-
-	/**
-	 * Adds the input vertex to the graph. If the vertex already
-	 * exists in the graph, it will not be added again.
-	 * 
-	 * @param vertex the vertex to be added
-	 * @return the new graph containing the existing vertices as well as the one just added
-	 */
-	public Graph<K, VV, EV> addVertex(final Vertex<K, VV> vertex) {
-		List<Vertex<K, VV>> newVertex = new ArrayList<Vertex<K, VV>>();
-		newVertex.add(vertex);
-
-		return addVertices(newVertex);
-	}
-
-	/**
-	 * Adds the list of vertices, passed as input, to the graph.
-	 * If the vertices already exist in the graph, they will not be added once more.
-	 *
-	 * @param verticesToAdd the list of vertices to add
-	 * @return the new graph containing the existing and newly added vertices
-	 */
-	public Graph<K, VV, EV> addVertices(List<Vertex<K, VV>> verticesToAdd) {
-		// Add the vertices
-		DataSet<Vertex<K, VV>> newVertices = this.vertices.union(this.context.fromCollection(verticesToAdd)).distinct();
-
-		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
-	}
-
-	/**
-	 * Adds the given edge to the graph. If the source and target vertices do
-	 * not exist in the graph, they will also be added.
-	 * 
-	 * @param source the source vertex of the edge
-	 * @param target the target vertex of the edge
-	 * @param edgeValue the edge value
-	 * @return the new graph containing the existing vertices and edges plus the
-	 *         newly added edge
-	 */
-	public Graph<K, VV, EV> addEdge(Vertex<K, VV> source, Vertex<K, VV> target, EV edgeValue) {
-		Graph<K, VV, EV> partialGraph = fromCollection(Arrays.asList(source, target),
-				Arrays.asList(new Edge<K, EV>(source.f0, target.f0, edgeValue)),
-				this.context);
-		return this.union(partialGraph);
-	}
-
-	/**
-	 * Adds the given list edges to the graph.
-	 *
-	 * When adding an edge for a non-existing set of vertices, the edge is considered invalid and ignored.
-	 *
-	 * @param newEdges the data set of edges to be added
-	 * @return a new graph containing the existing edges plus the newly added edges.
-	 */
-	public Graph<K, VV, EV> addEdges(List<Edge<K, EV>> newEdges) {
-
-		DataSet<Edge<K,EV>> newEdgesDataSet = this.context.fromCollection(newEdges);
-
-		DataSet<Edge<K,EV>> validNewEdges = this.getVertices().join(newEdgesDataSet)
-				.where(0).equalTo(0)
-				.with(new JoinVerticesWithEdgesOnSrc<K, VV, EV>())
-				.join(this.getVertices()).where(1).equalTo(0)
-				.with(new JoinWithVerticesOnTrg<K, VV, EV>());
-
-		return Graph.fromDataSet(this.vertices, this.edges.union(validNewEdges), this.context);
-	}
-
-	@ForwardedFieldsSecond("f0; f1; f2")
-	private static final class JoinVerticesWithEdgesOnSrc<K, VV, EV> implements
-			JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
-
-		@Override
-		public Edge<K, EV> join(Vertex<K, VV> vertex, Edge<K, EV> edge) throws Exception {
-			return edge;
-		}
-	}
-
-	@ForwardedFieldsFirst("f0; f1; f2")
-	private static final class JoinWithVerticesOnTrg<K, VV, EV> implements
-			JoinFunction<Edge<K, EV>, Vertex<K, VV>, Edge<K, EV>> {
-
-		@Override
-		public Edge<K, EV> join(Edge<K, EV> edge, Vertex<K, VV> vertex) throws Exception {
-			return edge;
-		}
-	}
-
-	/**
-	 * Removes the given vertex and its edges from the graph.
-	 * 
-	 * @param vertex the vertex to remove
-	 * @return the new graph containing the existing vertices and edges without
-	 *         the removed vertex and its edges
-	 */
-	public Graph<K, VV, EV> removeVertex(Vertex<K, VV> vertex) {
-
-		List<Vertex<K, VV>> vertexToBeRemoved = new ArrayList<Vertex<K, VV>>();
-		vertexToBeRemoved.add(vertex);
-
-		return removeVertices(vertexToBeRemoved);
-	}
-	/**
-	 * Removes the given list of vertices and its edges from the graph.
-	 *
-	 * @param verticesToBeRemoved the list of vertices to be removed
-	 * @return the resulted graph containing the initial vertices and edges minus the vertices
-	 * 		   and edges removed.
-	 */
-
-	public Graph<K, VV, EV> removeVertices(List<Vertex<K, VV>> verticesToBeRemoved)
-	{
-		return removeVertices(this.context.fromCollection(verticesToBeRemoved));
-	}
-
-	/**
-	 * Removes the given list of vertices and its edges from the graph.
-	 *
-	 * @param verticesToBeRemoved the DataSet of vertices to be removed
-	 * @return the resulted graph containing the initial vertices and edges minus the vertices
-	 * 		   and edges removed.
-	 */
-	private Graph<K, VV, EV> removeVertices(DataSet<Vertex<K, VV>> verticesToBeRemoved) {
-
-		DataSet<Vertex<K, VV>> newVertices = getVertices().coGroup(verticesToBeRemoved).where(0).equalTo(0)
-				.with(new VerticesRemovalCoGroup<K, VV>());
-
-		DataSet < Edge < K, EV >> newEdges = newVertices.join(getEdges()).where(0).equalTo(0)
-				// if the edge source was removed, the edge will also be removed
-				.with(new ProjectEdgeToBeRemoved<K, VV, EV>())
-				// if the edge target was removed, the edge will also be removed
-				.join(newVertices).where(1).equalTo(0)
-				.with(new ProjectEdge<K, VV, EV>());
-
-		return new Graph<K, VV, EV>(newVertices, newEdges, context);
-	}
-
-	private static final class VerticesRemovalCoGroup<K, VV> implements CoGroupFunction<Vertex<K, VV>, Vertex<K, VV>, Vertex<K, VV>> {
-
-		@Override
-		public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Vertex<K, VV>> vertexToBeRemoved,
-							Collector<Vertex<K, VV>> out) throws Exception {
-
-			final Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
-			final Iterator<Vertex<K, VV>> vertexToBeRemovedIterator = vertexToBeRemoved.iterator();
-			Vertex<K, VV> next;
-
-			if (vertexIterator.hasNext()) {
-				if (!vertexToBeRemovedIterator.hasNext()) {
-					next = vertexIterator.next();
-					out.collect(next);
-				}
-			}
-		}
-	}
-
-
-
-	@ForwardedFieldsSecond("f0; f1; f2")
-	private static final class ProjectEdgeToBeRemoved<K,VV,EV> implements JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
-		@Override
-		public Edge<K, EV> join(Vertex<K, VV> vertex, Edge<K, EV> edge) throws Exception {
-			return edge;
-		}
-	}
-
-	 /**
-	 * Removes all edges that match the given edge from the graph.
-	 * 
-	 * @param edge the edge to remove
-	 * @return the new graph containing the existing vertices and edges without
-	 *         the removed edges
-	 */
-	public Graph<K, VV, EV> removeEdge(Edge<K, EV> edge) {
-		DataSet<Edge<K, EV>> newEdges = getEdges().filter(new EdgeRemovalEdgeFilter<K, EV>(edge));
-		return new Graph<K, VV, EV>(this.vertices, newEdges, this.context);
-	}
-
-	private static final class EdgeRemovalEdgeFilter<K, EV>
-			implements FilterFunction<Edge<K, EV>> {
-		private Edge<K, EV> edgeToRemove;
-
-		public EdgeRemovalEdgeFilter(Edge<K, EV> edge) {
-			edgeToRemove = edge;
-		}
-
-		@Override
-		public boolean filter(Edge<K, EV> edge) {
-			return (!(edge.f0.equals(edgeToRemove.f0) && edge.f1
-					.equals(edgeToRemove.f1)));
-		}
-	}
-
-	/**
-	 * Removes all the edges that match the edges in the given data set from the graph.
-	 *
-	 * @param edgesToBeRemoved the list of edges to be removed
-	 * @return a new graph where the edges have been removed and in which the vertices remained intact
-	 */
-	public Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved) {
-
-		DataSet<Edge<K, EV>> newEdges = getEdges().coGroup(this.context.fromCollection(edgesToBeRemoved))
-				.where(0,1).equalTo(0,1).with(new EdgeRemovalCoGroup<K, EV>());
-
-		return new Graph<K, VV, EV>(this.vertices, newEdges, context);
-	}
-
-	private static final class EdgeRemovalCoGroup<K,EV> implements CoGroupFunction<Edge<K, EV>, Edge<K, EV>, Edge<K, EV>> {
-
-		@Override
-		public void coGroup(Iterable<Edge<K, EV>> edge, Iterable<Edge<K, EV>> edgeToBeRemoved,
-							Collector<Edge<K, EV>> out) throws Exception {
-
-			final Iterator<Edge<K, EV>> edgeIterator = edge.iterator();
-			final Iterator<Edge<K, EV>> edgeToBeRemovedIterator = edgeToBeRemoved.iterator();
-			Edge<K, EV> next;
-
-			if (edgeIterator.hasNext()) {
-				if (!edgeToBeRemovedIterator.hasNext()) {
-					next = edgeIterator.next();
-					out.collect(next);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Performs union on the vertices and edges sets of the input graphs
-	 * removing duplicate vertices but maintaining duplicate edges.
-	 * 
-	 * @param graph the graph to perform union with
-	 * @return a new graph
-	 */
-	public Graph<K, VV, EV> union(Graph<K, VV, EV> graph) {
-
-		DataSet<Vertex<K, VV>> unionedVertices = graph.getVertices().union(this.getVertices()).distinct();
-		DataSet<Edge<K, EV>> unionedEdges = graph.getEdges().union(this.getEdges());
-		return new Graph<K, VV, EV>(unionedVertices, unionedEdges, this.context);
-	}
-
-	/**
-	 * Performs Difference on the vertex and edge sets of the input graphs
-	 * removes common vertices and edges. If a source/target vertex is removed, its corresponding edge will also be removed
-	 * @param graph the graph to perform difference with
-	 * @return a new graph where the common vertices and edges have been removed
-	 */
-	public Graph<K,VV,EV> difference(Graph<K,VV,EV> graph) {
-		DataSet<Vertex<K,VV>> removeVerticesData = graph.getVertices();
-		return this.removeVertices(removeVerticesData);
-	}
-
-	/**
-	 * Runs a Vertex-Centric iteration on the graph.
-	 * No configuration options are provided.
-	 *
-	 * @param vertexUpdateFunction the vertex update function
-	 * @param messagingFunction the messaging function
-	 * @param maximumNumberOfIterations maximum number of iterations to perform
-	 * 
-	 * @return the updated Graph after the vertex-centric iteration has converged or
-	 * after maximumNumberOfIterations.
-	 */
-	public <M> Graph<K, VV, EV> runVertexCentricIteration(
-			VertexUpdateFunction<K, VV, M> vertexUpdateFunction,
-			MessagingFunction<K, VV, M, EV> messagingFunction,
-			int maximumNumberOfIterations) {
-
-		return this.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
-				maximumNumberOfIterations, null);
-	}
-
-	/**
-	 * Runs a Vertex-Centric iteration on the graph with configuration options.
-	 * 
-	 * @param vertexUpdateFunction the vertex update function
-	 * @param messagingFunction the messaging function
-	 * @param maximumNumberOfIterations maximum number of iterations to perform
-	 * @param parameters the iteration configuration parameters
-	 * 
-	 * @return the updated Graph after the vertex-centric iteration has converged or
-	 * after maximumNumberOfIterations.
-	 */
-	public <M> Graph<K, VV, EV> runVertexCentricIteration(
-			VertexUpdateFunction<K, VV, M> vertexUpdateFunction,
-			MessagingFunction<K, VV, M, EV> messagingFunction,
-			int maximumNumberOfIterations, VertexCentricConfiguration parameters) {
-
-		VertexCentricIteration<K, VV, M, EV> iteration = VertexCentricIteration.withEdges(
-				edges, vertexUpdateFunction, messagingFunction, maximumNumberOfIterations);
-
-		iteration.configure(parameters);
-
-		DataSet<Vertex<K, VV>> newVertices = this.getVertices().runOperation(iteration);
-
-		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
-	}
-
-	/**
-	 * Runs a Gather-Sum-Apply iteration on the graph.
-	 * No configuration options are provided.
-	 *
-	 * @param gatherFunction the gather function collects information about adjacent vertices and edges
-	 * @param sumFunction the sum function aggregates the gathered information
-	 * @param applyFunction the apply function updates the vertex values with the aggregates
-	 * @param maximumNumberOfIterations maximum number of iterations to perform
-	 * @param <M> the intermediate type used between gather, sum and apply
-	 *
-	 * @return the updated Graph after the gather-sum-apply iteration has converged or
-	 * after maximumNumberOfIterations.
-	 */
-	public <M> Graph<K, VV, EV> runGatherSumApplyIteration(
-			GatherFunction<VV, EV, M> gatherFunction, SumFunction<VV, EV, M> sumFunction,
-			ApplyFunction<K, VV, M> applyFunction, int maximumNumberOfIterations) {
-
-		return this.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
-				maximumNumberOfIterations, null);
-	}
-
-	/**
-	 * Runs a Gather-Sum-Apply iteration on the graph with configuration options.
-	 *
-	 * @param gatherFunction the gather function collects information about adjacent vertices and edges
-	 * @param sumFunction the sum function aggregates the gathered information
-	 * @param applyFunction the apply function updates the vertex values with the aggregates
-	 * @param maximumNumberOfIterations maximum number of iterations to perform
-	 * @param parameters the iteration configuration parameters
-	 * @param <M> the intermediate type used between gather, sum and apply
-	 *
-	 * @return the updated Graph after the gather-sum-apply iteration has converged or
-	 * after maximumNumberOfIterations.
-	 */
-	public <M> Graph<K, VV, EV> runGatherSumApplyIteration(
-			GatherFunction<VV, EV, M> gatherFunction, SumFunction<VV, EV, M> sumFunction,
-			ApplyFunction<K, VV, M> applyFunction, int maximumNumberOfIterations,
-			GSAConfiguration parameters) {
-
-		GatherSumApplyIteration<K, VV, EV, M> iteration = GatherSumApplyIteration.withEdges(
-				edges, gatherFunction, sumFunction, applyFunction, maximumNumberOfIterations);
-
-		iteration.configure(parameters);
-
-		DataSet<Vertex<K, VV>> newVertices = vertices.runOperation(iteration);
-
-		return new Graph<K, VV, EV>(newVertices, this.edges, this.context);
-	}
-
-	/**
-	 * @param algorithm the algorithm to run on the Graph
-	 * @param <T> the return type
-	 * @return the result of the graph algorithm
-	 * @throws Exception
-	 */
-	public <T> T run(GraphAlgorithm<K, VV, EV, T> algorithm) throws Exception {
-		return algorithm.run(this);
-	}
-
-	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors has access to the vertex
-	 * value.
-	 * 
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
-	 * @param <T> the output type
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction) throws IllegalArgumentException {
-		switch (direction) {
-		case IN:
-			// create <edge-sourceVertex> pairs
-			DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
-					.join(this.vertices).where(0).equalTo(0);
-			return vertices.coGroup(edgesWithSources)
-					.where(0).equalTo("f0.f1")
-					.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction));
-		case OUT:
-			// create <edge-targetVertex> pairs
-			DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
-					.join(this.vertices).where(1).equalTo(0);
-			return vertices.coGroup(edgesWithTargets)
-					.where(0).equalTo("f0.f0")
-					.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction));
-		case ALL:
-			// create <edge-sourceOrTargetVertex> pairs
-			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
-					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
-
-			return vertices.coGroup(edgesWithNeighbors)
-					.where(0).equalTo(0)
-					.with(new ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>(neighborsFunction));
-		default:
-			throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors has access to the vertex
-	 * value.
-	 *
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
-	 * @param <T> the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
-		switch (direction) {
-			case IN:
-				// create <edge-sourceVertex> pairs
-				DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
-						.join(this.vertices).where(0).equalTo(0);
-				return vertices.coGroup(edgesWithSources)
-						.where(0).equalTo("f0.f1")
-						.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			case OUT:
-				// create <edge-targetVertex> pairs
-				DataSet<Tuple2<Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
-						.join(this.vertices).where(1).equalTo(0);
-				return vertices.coGroup(edgesWithTargets)
-						.where(0).equalTo("f0.f0")
-						.with(new ApplyNeighborCoGroupFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			case ALL:
-				// create <edge-sourceOrTargetVertex> pairs
-				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
-						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
-
-				return vertices.coGroup(edgesWithNeighbors)
-						.where(0).equalTo(0)
-						.with(new ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-
-	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors only has access to the
-	 * vertex id (not the vertex value).
-	 * 
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
-	 * @param <T> the output type
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction) throws IllegalArgumentException {
-		switch (direction) {
-		case IN:
-			// create <edge-sourceVertex> pairs
-			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
-					.join(this.vertices).where(0).equalTo(0)
-					.with(new ProjectVertexIdJoin<K, VV, EV>(1))
-					.withForwardedFieldsFirst("f1->f0");
-			return edgesWithSources.groupBy(0).reduceGroup(
-					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
-		case OUT:
-			// create <edge-targetVertex> pairs
-			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
-					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectVertexIdJoin<K, VV, EV>(0))
-					.withForwardedFieldsFirst("f0");
-			return edgesWithTargets.groupBy(0).reduceGroup(
-					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
-		case ALL:
-			// create <edge-sourceOrTargetVertex> pairs
-			DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-					.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
-					.join(this.vertices).where(1).equalTo(0)
-					.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
-
-			return edgesWithNeighbors.groupBy(0).reduceGroup(
-					new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction));
-		default:
-			throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the neighbors (edges and vertices) of each
-	 * vertex. The function applied on the neighbors only has access to the
-	 * vertex id (not the vertex value).
-	 *
-	 * @param neighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
-	 * @param <T> the output type
-	 * @param typeInfo the explicit return type.
-	 * @return a dataset of a T
-	 * @throws IllegalArgumentException
-	 */
-	public <T> DataSet<T> groupReduceOnNeighbors(NeighborsFunction<K, VV, EV, T> neighborsFunction,
-												EdgeDirection direction, TypeInformation<T> typeInfo) throws IllegalArgumentException {
-		switch (direction) {
-			case IN:
-				// create <edge-sourceVertex> pairs
-				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithSources = edges
-						.join(this.vertices).where(0).equalTo(0)
-						.with(new ProjectVertexIdJoin<K, VV, EV>(1))
-						.withForwardedFieldsFirst("f1->f0");
-				return edgesWithSources.groupBy(0).reduceGroup(
-						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			case OUT:
-				// create <edge-targetVertex> pairs
-				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithTargets = edges
-						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectVertexIdJoin<K, VV, EV>(0))
-						.withForwardedFieldsFirst("f0");
-				return edgesWithTargets.groupBy(0).reduceGroup(
-						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			case ALL:
-				// create <edge-sourceOrTargetVertex> pairs
-				DataSet<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edgesWithNeighbors = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
-						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectEdgeWithNeighbor<K, VV, EV>());
-
-				return edgesWithNeighbors.groupBy(0).reduceGroup(
-						new ApplyNeighborGroupReduceFunction<K, VV, EV, T>(neighborsFunction)).returns(typeInfo);
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	private static final class ApplyNeighborGroupReduceFunction<K, VV, EV, T>
-			implements GroupReduceFunction<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
-
-		private NeighborsFunction<K, VV, EV, T> function;
-
-		public ApplyNeighborGroupReduceFunction(NeighborsFunction<K, VV, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void reduce(Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> edges, Collector<T> out) throws Exception {
-			function.iterateNeighbors(edges, out);
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(NeighborsFunction.class, function.getClass(), 3, null, null);
-		}
-	}
-
-	@ForwardedFieldsSecond("f1")
-	private static final class ProjectVertexWithNeighborValueJoin<K, VV, EV>
-			implements FlatJoinFunction<Edge<K, EV>, Vertex<K, VV>, Tuple2<K, VV>> {
-
-		private int fieldPosition;
-
-		public ProjectVertexWithNeighborValueJoin(int position) {
-			this.fieldPosition = position;
-		}
-
-		@SuppressWarnings("unchecked")
-		public void join(Edge<K, EV> edge, Vertex<K, VV> otherVertex, 
-				Collector<Tuple2<K, VV>> out) {
-			out.collect(new Tuple2<K, VV>((K) edge.getField(fieldPosition), otherVertex.getValue()));
-		}
-	}
-
-	private static final class ProjectVertexIdJoin<K, VV, EV> implements FlatJoinFunction<
-		Edge<K, EV>, Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> {
-
-		private int fieldPosition;
-
-		public ProjectVertexIdJoin(int position) {
-			this.fieldPosition = position;
-		}
-
-		@SuppressWarnings("unchecked")
-		public void join(Edge<K, EV> edge, Vertex<K, VV> otherVertex,
-						Collector<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> out) {
-			out.collect(new Tuple3<K, Edge<K, EV>, Vertex<K, VV>>((K) edge.getField(fieldPosition), edge, otherVertex));
-		}
-	}
-
-	@ForwardedFieldsFirst("f0")
-	@ForwardedFieldsSecond("f1")
-	private static final class ProjectNeighborValue<K, VV, EV> implements FlatJoinFunction<
-		Tuple3<K, K, Edge<K, EV>>, Vertex<K, VV>, Tuple2<K, VV>> {
-
-		public void join(Tuple3<K, K, Edge<K, EV>> keysWithEdge, Vertex<K, VV> neighbor,
-				Collector<Tuple2<K, VV>> out) {
-
-			out.collect(new Tuple2<K, VV>(keysWithEdge.f0, neighbor.getValue()));
-		}
-	}
-
-	@ForwardedFieldsFirst("f0; f2->f1")
-	@ForwardedFieldsSecond("*->f2")
-	private static final class ProjectEdgeWithNeighbor<K, VV, EV> implements FlatJoinFunction<
-		Tuple3<K, K, Edge<K, EV>>, Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> {
-
-		public void join(Tuple3<K, K, Edge<K, EV>> keysWithEdge, Vertex<K, VV> neighbor,
-						Collector<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> out) {
-			out.collect(new Tuple3<K, Edge<K, EV>, Vertex<K, VV>>(keysWithEdge.f0, keysWithEdge.f2, neighbor));
-		}
-	}
-
-	private static final class ApplyNeighborCoGroupFunction<K, VV, EV, T> implements CoGroupFunction<
-		Vertex<K, VV>, Tuple2<Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
-
-		private NeighborsFunctionWithVertexValue<K, VV, EV, T> function;
-
-		public ApplyNeighborCoGroupFunction(NeighborsFunctionWithVertexValue<K, VV, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors,
-				Collector<T> out) throws Exception {
-			function.iterateNeighbors(vertex.iterator().next(),	neighbors, out);
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class,	function.getClass(), 3, null, null);
-		}
-	}
-
-	private static final class ApplyCoGroupFunctionOnAllNeighbors<K, VV, EV, T>
-			implements CoGroupFunction<Vertex<K, VV>, Tuple3<K, Edge<K, EV>, Vertex<K, VV>>, T>, ResultTypeQueryable<T> {
-
-		private NeighborsFunctionWithVertexValue<K, VV, EV, T> function;
-
-		public ApplyCoGroupFunctionOnAllNeighbors(NeighborsFunctionWithVertexValue<K, VV, EV, T> fun) {
-			this.function = fun;
-		}
-
-		public void coGroup(Iterable<Vertex<K, VV>> vertex,
-				final Iterable<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> keysWithNeighbors, 
-				Collector<T> out) throws Exception {
-
-			final Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighborsIterator = new Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>>() {
-
-				final Iterator<Tuple3<K, Edge<K, EV>, Vertex<K, VV>>> keysWithEdgesIterator = keysWithNeighbors.iterator();
-
-				@Override
-				public boolean hasNext() {
-					return keysWithEdgesIterator.hasNext();
-				}
-
-				@Override
-				public Tuple2<Edge<K, EV>, Vertex<K, VV>> next() {
-					Tuple3<K, Edge<K, EV>, Vertex<K, VV>> next = keysWithEdgesIterator.next();
-					return new Tuple2<Edge<K, EV>, Vertex<K, VV>>(next.f1, next.f2);
-				}
-
-				@Override
-				public void remove() {
-					keysWithEdgesIterator.remove();
-				}
-			};
-
-			Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighborsIterable = new Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>>() {
-				public Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>> iterator() {
-					return neighborsIterator;
-				}
-			};
-
-			Iterator<Vertex<K, VV>> vertexIterator = vertex.iterator();
-
-			if (vertexIterator.hasNext()) {
-				function.iterateNeighbors(vertexIterator.next(), neighborsIterable, out);
-			} else {
-				throw new NoSuchElementException("The edge src/trg id could not be found within the vertexIds");
-			}
-		}
-
-		@Override
-		public TypeInformation<T> getProducedType() {
-			return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class,	function.getClass(), 3, null, null);
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the neighbor values of each
-	 * vertex.
-	 *
-	 * @param reduceNeighborsFunction the function to apply to the neighborhood
-	 * @param direction the edge direction (in-, out-, all-)
-	 * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
-	 * @throws IllegalArgumentException
-	 */
-	public DataSet<Tuple2<K, VV>> reduceOnNeighbors(ReduceNeighborsFunction<VV> reduceNeighborsFunction,
-									EdgeDirection direction) throws IllegalArgumentException {
-		switch (direction) {
-			case IN:
-				// create <vertex-source value> pairs
-				final DataSet<Tuple2<K, VV>> verticesWithSourceNeighborValues = edges
-						.join(this.vertices).where(0).equalTo(0)
-						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(1))
-						.withForwardedFieldsFirst("f1->f0");
-				return verticesWithSourceNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction));
-			case OUT:
-				// create <vertex-target value> pairs
-				DataSet<Tuple2<K, VV>> verticesWithTargetNeighborValues = edges
-						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectVertexWithNeighborValueJoin<K, VV, EV>(0))
-						.withForwardedFieldsFirst("f0");
-				return verticesWithTargetNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction));
-			case ALL:
-				// create <vertex-neighbor value> pairs
-				DataSet<Tuple2<K, VV>> verticesWithNeighborValues = edges
-						.flatMap(new EmitOneEdgeWithNeighborPerNode<K, EV>())
-						.join(this.vertices).where(1).equalTo(0)
-						.with(new ProjectNeighborValue<K, VV, EV>());
-
-				return verticesWithNeighborValues.groupBy(0).reduce(new ApplyNeighborReduceFunction<K, VV>(
-						reduceNeighborsFunction));
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	@ForwardedFields("f0")
-	private static final class ApplyNeighborReduceFunction<K, VV> implements ReduceFunction<Tuple2<K, VV>> {
-
-		private ReduceNeighborsFunction<VV> function;
-
-		public ApplyNeighborReduceFunction(ReduceNeighborsFunction<VV> fun) {
-			this.function = fun;
-		}
-
-		@Override
-		public Tuple2<K, VV> reduce(Tuple2<K, VV> first, Tuple2<K, VV> second) throws Exception {
-			first.setField(function.reduceNeighbors(first.f1, second.f1), 1);
-			return first;
-		}
-	}
-
-	/**
-	 * Compute an aggregate over the edge values of each vertex.
-	 *
-	 * @param reduceEdgesFunction
-	 *            the function to apply to the neighborhood
-	 * @param direction
-	 *            the edge direction (in-, out-, all-)
-	 * @return a Dataset containing one value per vertex(vertex key, aggregate edge value)
-	 * @throws IllegalArgumentException
-	 */
-	public DataSet<Tuple2<K, EV>> reduceOnEdges(ReduceEdgesFunction<EV> reduceEdgesFunction,
-								EdgeDirection direction) throws IllegalArgumentException {
-
-		switch (direction) {
-			case IN:
-				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(1))
-						.withForwardedFields("f1->f0")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
-			case OUT:
-				return edges.map(new ProjectVertexWithEdgeValueMap<K, EV>(0))
-						.withForwardedFields("f0->f0")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
-			case ALL:
-				return edges.flatMap(new EmitOneVertexWithEdgeValuePerNode<K, EV>())
-						.withForwardedFields("f2->f1")
-						.groupBy(0).reduce(new ApplyReduceFunction<K, EV>(reduceEdgesFunction));
-			default:
-				throw new IllegalArgumentException("Illegal edge direction");
-		}
-	}
-
-	@ForwardedFields("f0")
-	private static final class ApplyReduceFunction<K, EV> implements ReduceFunction<Tuple2<K, EV>> {
-
-		private ReduceEdgesFunction<EV> function;
-
-		public ApplyReduceFunction(ReduceEdgesFunction<EV> fun) {
-			this.function = fun;
-		}
-
-		@Override
-		public Tuple2<K, EV> reduce(Tuple2<K, EV> first, Tuple2<K, EV> second) throws Exception {
-			first.setField(function.reduceEdges(first.f1, second.f1), 1);
-			return first;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
deleted file mode 100644
index 08cf011..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/GraphAlgorithm.java
+++ /dev/null
@@ -1,30 +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;
-
-/**
- * @param <K> key type
- * @param <VV> vertex value type
- * @param <EV> edge value type
- * @param <T> the return type
- */
-public interface GraphAlgorithm<K, VV, EV, T> {
-
-	public T run(Graph<K, VV, EV> input) throws Exception;
-}


[07/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
deleted file mode 100755
index 23ccb68..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherSumApplyIteration.java
+++ /dev/null
@@ -1,425 +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.gsa;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.RichFlatJoinFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
-import org.apache.flink.api.java.operators.CustomUnaryOperation;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.operators.JoinOperator;
-import org.apache.flink.api.java.operators.MapOperator;
-import org.apache.flink.api.java.operators.ReduceOperator;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.util.Collector;
-import java.util.Map;
-
-import com.google.common.base.Preconditions;
-
-/**
- * This class represents iterative graph computations, programmed in a gather-sum-apply perspective.
- *
- * @param <K> The type of the vertex key in the graph
- * @param <VV> The type of the vertex value in the graph
- * @param <EV> The type of the edge value in the graph
- * @param <M> The intermediate type used by the gather, sum and apply functions
- */
-public class GatherSumApplyIteration<K, VV, EV, M> implements CustomUnaryOperation<Vertex<K, VV>,
-		Vertex<K, VV>> {
-
-	private DataSet<Vertex<K, VV>> vertexDataSet;
-	private DataSet<Edge<K, EV>> edgeDataSet;
-
-	private final GatherFunction<VV, EV, M> gather;
-	private final SumFunction<VV, EV, M> sum;
-	private final ApplyFunction<K, VV, M> apply;
-	private final int maximumNumberOfIterations;
-	private EdgeDirection direction = EdgeDirection.OUT;
-
-	private GSAConfiguration configuration;
-
-	// ----------------------------------------------------------------------------------
-
-	private GatherSumApplyIteration(GatherFunction<VV, EV, M> gather, SumFunction<VV, EV, M> sum,
-			ApplyFunction<K, VV, M> apply, DataSet<Edge<K, EV>> edges, int maximumNumberOfIterations) {
-
-		Preconditions.checkNotNull(gather);
-		Preconditions.checkNotNull(sum);
-		Preconditions.checkNotNull(apply);
-		Preconditions.checkNotNull(edges);
-		Preconditions.checkArgument(maximumNumberOfIterations > 0, "The maximum number of iterations must be at least one.");
-
-		this.gather = gather;
-		this.sum = sum;
-		this.apply = apply;
-		this.edgeDataSet = edges;
-		this.maximumNumberOfIterations = maximumNumberOfIterations;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Custom Operator behavior
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Sets the input data set for this operator. In the case of this operator this input data set represents
-	 * the set of vertices with their initial state.
-	 *
-	 * @param dataSet The input data set, which in the case of this operator represents the set of
-	 *                vertices with their initial state.
-	 */
-	@Override
-	public void setInput(DataSet<Vertex<K, VV>> dataSet) {
-		this.vertexDataSet = dataSet;
-	}
-
-	/**
-	 * Computes the results of the gather-sum-apply iteration
-	 *
-	 * @return The resulting DataSet
-	 */
-	@Override
-	public DataSet<Vertex<K, VV>> createResult() {
-		if (vertexDataSet == null) {
-			throw new IllegalStateException("The input data set has not been set.");
-		}
-
-		// Prepare type information
-		TypeInformation<K> keyType = ((TupleTypeInfo<?>) vertexDataSet.getType()).getTypeAt(0);
-		TypeInformation<M> messageType = TypeExtractor.createTypeInfo(GatherFunction.class, gather.getClass(), 2, null, null);
-		TypeInformation<Tuple2<K, M>> innerType = new TupleTypeInfo<Tuple2<K, M>>(keyType, messageType);
-		TypeInformation<Vertex<K, VV>> outputType = vertexDataSet.getType();
-
-		// create a graph
-		Graph<K, VV, EV> graph =
-				Graph.fromDataSet(vertexDataSet, edgeDataSet, vertexDataSet.getExecutionEnvironment());
-
-		// check whether the numVertices option is set and, if so, compute the total number of vertices
-		// and set it within the gather, sum and apply functions
-		if (this.configuration != null && this.configuration.isOptNumVertices()) {
-			try {
-				long numberOfVertices = graph.numberOfVertices();
-				gather.setNumberOfVertices(numberOfVertices);
-				sum.setNumberOfVertices(numberOfVertices);
-				apply.setNumberOfVertices(numberOfVertices);
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
-
-		// Prepare UDFs
-		GatherUdf<K, VV, EV, M> gatherUdf = new GatherUdf<K, VV, EV, M>(gather, innerType);
-		SumUdf<K, VV, EV, M> sumUdf = new SumUdf<K, VV, EV, M>(sum, innerType);
-		ApplyUdf<K, VV, EV, M> applyUdf = new ApplyUdf<K, VV, EV, M>(apply, outputType);
-
-		final int[] zeroKeyPos = new int[] {0};
-		final DeltaIteration<Vertex<K, VV>, Vertex<K, VV>> iteration =
-				vertexDataSet.iterateDelta(vertexDataSet, maximumNumberOfIterations, zeroKeyPos);
-
-		// set up the iteration operator
-		if (this.configuration != null) {
-
-			iteration.name(this.configuration.getName(
-					"Gather-sum-apply iteration (" + gather + " | " + sum + " | " + apply + ")"));
-			iteration.parallelism(this.configuration.getParallelism());
-			iteration.setSolutionSetUnManaged(this.configuration.isSolutionSetUnmanagedMemory());
-
-			// register all aggregators
-			for (Map.Entry<String, Aggregator<?>> entry : this.configuration.getAggregators().entrySet()) {
-				iteration.registerAggregator(entry.getKey(), entry.getValue());
-			}
-		}
-		else {
-			// no configuration provided; set default name
-			iteration.name("Gather-sum-apply iteration (" + gather + " | " + sum + " | " + apply + ")");
-		}
-
-		// Prepare the neighbors
-		if(this.configuration != null) {
-			direction = this.configuration.getDirection();
-		}
-		DataSet<Tuple2<K, Neighbor<VV, EV>>> neighbors;
-		switch(direction) {
-			case OUT:
-				neighbors = iteration
-				.getWorkset().join(edgeDataSet)
-				.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
-				break;
-			case IN:
-				neighbors = iteration
-				.getWorkset().join(edgeDataSet)
-				.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>());
-				break;
-			case ALL:
-				neighbors =  iteration
-						.getWorkset().join(edgeDataSet)
-						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>()).union(iteration
-								.getWorkset().join(edgeDataSet)
-								.where(0).equalTo(1).with(new ProjectKeyWithNeighborIN<K, VV, EV>()));
-				break;
-			default:
-				neighbors = iteration
-						.getWorkset().join(edgeDataSet)
-						.where(0).equalTo(0).with(new ProjectKeyWithNeighborOUT<K, VV, EV>());
-				break;
-		}
-
-		// Gather, sum and apply
-		MapOperator<Tuple2<K, Neighbor<VV, EV>>, Tuple2<K, M>> gatherMapOperator = neighbors.map(gatherUdf);
-
-		// configure map gather function with name and broadcast variables
-		gatherMapOperator = gatherMapOperator.name("Gather");
-
-		if (this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getGatherBcastVars()) {
-				gatherMapOperator = gatherMapOperator.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-		DataSet<Tuple2<K, M>> gatheredSet = gatherMapOperator;
-
-		ReduceOperator<Tuple2<K, M>> sumReduceOperator = gatheredSet.groupBy(0).reduce(sumUdf);
-
-		// configure reduce sum function with name and broadcast variables
-		sumReduceOperator = sumReduceOperator.name("Sum");
-
-		if (this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getSumBcastVars()) {
-				sumReduceOperator = sumReduceOperator.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-		DataSet<Tuple2<K, M>> summedSet = sumReduceOperator;
-
-		JoinOperator<?, ?, Vertex<K, VV>> appliedSet = summedSet
-				.join(iteration.getSolutionSet())
-				.where(0)
-				.equalTo(0)
-				.with(applyUdf);
-
-		// configure join apply function with name and broadcast variables
-		appliedSet = appliedSet.name("Apply");
-
-		if (this.configuration != null) {
-			for (Tuple2<String, DataSet<?>> e : this.configuration.getApplyBcastVars()) {
-				appliedSet = appliedSet.withBroadcastSet(e.f1, e.f0);
-			}
-		}
-
-		// let the operator know that we preserve the key field
-		appliedSet.withForwardedFieldsFirst("0").withForwardedFieldsSecond("0");
-
-		return iteration.closeWith(appliedSet, appliedSet);
-	}
-
-	/**
-	 * Creates a new gather-sum-apply iteration operator for graphs
-	 *
-	 * @param edges The edge DataSet
-	 *
-	 * @param gather The gather function of the GSA iteration
-	 * @param sum The sum function of the GSA iteration
-	 * @param apply The apply function of the GSA iteration
-	 *
-	 * @param maximumNumberOfIterations The maximum number of iterations executed
-	 *
-	 * @param <K> The type of the vertex key in the graph
-	 * @param <VV> The type of the vertex value in the graph
-	 * @param <EV> The type of the edge value in the graph
-	 * @param <M> The intermediate type used by the gather, sum and apply functions
-	 *
-	 * @return An in stance of the gather-sum-apply graph computation operator.
-	 */
-	public static final <K, VV, EV, M> GatherSumApplyIteration<K, VV, EV, M>
-		withEdges(DataSet<Edge<K, EV>> edges, GatherFunction<VV, EV, M> gather,
-		SumFunction<VV, EV, M> sum, ApplyFunction<K, VV, M> apply, int maximumNumberOfIterations) {
-
-		return new GatherSumApplyIteration<K, VV, EV, M>(gather, sum, apply, edges, maximumNumberOfIterations);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Wrapping UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	@ForwardedFields("f0")
-	private static final class GatherUdf<K, VV, EV, M> extends RichMapFunction<Tuple2<K, Neighbor<VV, EV>>,
-			Tuple2<K, M>> implements ResultTypeQueryable<Tuple2<K, M>> {
-
-		private final GatherFunction<VV, EV, M> gatherFunction;
-		private transient TypeInformation<Tuple2<K, M>> resultType;
-
-		private GatherUdf(GatherFunction<VV, EV, M> gatherFunction, TypeInformation<Tuple2<K, M>> resultType) {
-			this.gatherFunction = gatherFunction;
-			this.resultType = resultType;
-		}
-
-		@Override
-		public Tuple2<K, M> map(Tuple2<K, Neighbor<VV, EV>> neighborTuple) {
-			M result = this.gatherFunction.gather(neighborTuple.f1);
-			return new Tuple2<K, M>(neighborTuple.f0, result);
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
-				this.gatherFunction.init(getIterationRuntimeContext());
-			}
-			this.gatherFunction.preSuperstep();
-		}
-
-		@Override
-		public void close() throws Exception {
-			this.gatherFunction.postSuperstep();
-		}
-
-		@Override
-		public TypeInformation<Tuple2<K, M>> getProducedType() {
-			return this.resultType;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumUdf<K, VV, EV, M> extends RichReduceFunction<Tuple2<K, M>>
-			implements ResultTypeQueryable<Tuple2<K, M>>{
-
-		private final SumFunction<VV, EV, M> sumFunction;
-		private transient TypeInformation<Tuple2<K, M>> resultType;
-
-		private SumUdf(SumFunction<VV, EV, M> sumFunction, TypeInformation<Tuple2<K, M>> resultType) {
-			this.sumFunction = sumFunction;
-			this.resultType = resultType;
-		}
-
-		@Override
-		public Tuple2<K, M> reduce(Tuple2<K, M> arg0, Tuple2<K, M> arg1) throws Exception {
-			K key = arg0.f0;
-			M result = this.sumFunction.sum(arg0.f1, arg1.f1);
-			return new Tuple2<K, M>(key, result);
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
-				this.sumFunction.init(getIterationRuntimeContext());
-			}
-			this.sumFunction.preSuperstep();
-		}
-
-		@Override
-		public void close() throws Exception {
-			this.sumFunction.postSuperstep();
-		}
-
-		@Override
-		public TypeInformation<Tuple2<K, M>> getProducedType() {
-			return this.resultType;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ApplyUdf<K, VV, EV, M> extends RichFlatJoinFunction<Tuple2<K, M>,
-			Vertex<K, VV>, Vertex<K, VV>> implements ResultTypeQueryable<Vertex<K, VV>> {
-
-		private final ApplyFunction<K, VV, M> applyFunction;
-		private transient TypeInformation<Vertex<K, VV>> resultType;
-
-		private ApplyUdf(ApplyFunction<K, VV, M> applyFunction, TypeInformation<Vertex<K, VV>> resultType) {
-			this.applyFunction = applyFunction;
-			this.resultType = resultType;
-		}
-
-		@Override
-		public void join(Tuple2<K, M> newValue, final Vertex<K, VV> currentValue, final Collector<Vertex<K, VV>> out) throws Exception {
-
-			this.applyFunction.setOutput(currentValue, out);
-			this.applyFunction.apply(newValue.f1, currentValue.getValue());
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
-				this.applyFunction.init(getIterationRuntimeContext());
-			}
-			this.applyFunction.preSuperstep();
-		}
-
-		@Override
-		public void close() throws Exception {
-			this.applyFunction.postSuperstep();
-		}
-
-		@Override
-		public TypeInformation<Vertex<K, VV>> getProducedType() {
-			return this.resultType;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	@ForwardedFieldsSecond("f1->f0")
-	private static final class ProjectKeyWithNeighborOUT<K, VV, EV> implements FlatJoinFunction<
-			Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Neighbor<VV, EV>>> {
-
-		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple2<K, Neighbor<VV, EV>>> out) {
-			out.collect(new Tuple2<K, Neighbor<VV, EV>>(
-					edge.getTarget(), new Neighbor<VV, EV>(vertex.getValue(), edge.getValue())));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	@ForwardedFieldsSecond({"f0"})
-	private static final class ProjectKeyWithNeighborIN<K, VV, EV> implements FlatJoinFunction<
-			Vertex<K, VV>, Edge<K, EV>, Tuple2<K, Neighbor<VV, EV>>> {
-
-		public void join(Vertex<K, VV> vertex, Edge<K, EV> edge, Collector<Tuple2<K, Neighbor<VV, EV>>> out) {
-			out.collect(new Tuple2<K, Neighbor<VV, EV>>(
-					edge.getSource(), new Neighbor<VV, EV>(vertex.getValue(), edge.getValue())));
-		}
-	}
-
-
-
-
-	/**
-	 * Configures this gather-sum-apply iteration with the provided parameters.
-	 *
-	 * @param parameters the configuration parameters
-	 */
-	public void configure(GSAConfiguration parameters) {
-		this.configuration = parameters;
-	}
-
-	/**
-	 * @return the configuration parameters of this gather-sum-apply iteration
-	 */
-	public GSAConfiguration getIterationConfiguration() {
-		return this.configuration;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
deleted file mode 100755
index 7fa1ed2..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
+++ /dev/null
@@ -1,45 +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.gsa;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
-/**
- * This class represents a <sourceVertex, edge> pair
- * This is a wrapper around Tuple2<VV, EV> for convenience in the GatherFunction
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- */
-@SuppressWarnings("serial")
-public class Neighbor<VV, EV> extends Tuple2<VV, EV> {
-
-	public Neighbor() {}
-
-	public Neighbor(VV neighborValue, EV edgeValue) {
-		super(neighborValue, edgeValue);
-	}
-
-	public VV getNeighborValue() {
-		return this.f0;
-	}
-
-	public EV getEdgeValue() {
-		return this.f1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
deleted file mode 100755
index f27e275..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
+++ /dev/null
@@ -1,134 +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.gsa;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.types.Value;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * The base class for the second step of a {@link GatherSumApplyIteration}.
- *
- * @param <VV> the vertex value type
- * @param <EV> the edge value type
- * @param <M> the output type
- */
-@SuppressWarnings("serial")
-public abstract class SumFunction<VV, EV, M> implements Serializable {
-
-	// --------------------------------------------------------------------------------------------
-	//  Attribute that allows access to the total number of vertices inside an iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private long numberOfVertices = -1L;
-
-	/**
-	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
-	 * option has been set; -1 otherwise.
-	 */
-	public long getNumberOfVertices() {
-		return numberOfVertices;
-	}
-
-	void setNumberOfVertices(long numberOfVertices) {
-		this.numberOfVertices = numberOfVertices;
-	}
-
-	//---------------------------------------------------------------------------------------------
-	/**
-	 * This method is invoked once per superstep, after the {@link GatherFunction} 
-	 * in a {@link GatherSumApplyIteration}.
-	 * It combines the partial values produced by {@link GatherFunction#gather(Neighbor)}
-	 * in pairs, until a single value has been computed.
-	 * 
-	 * @param arg0 the first partial value.
-	 * @param arg1 the second partial value.
-	 * @return the combined value.
-	 */
-	public abstract M sum(M arg0, M arg1);
-
-	/**
-	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
-	 */
-	public void preSuperstep() {}
-
-	/**
-	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
-	 */
-	public void postSuperstep() {}
-
-	/**
-	 * Gets the number of the superstep, starting at <tt>1</tt>.
-	 *
-	 * @return The number of the current superstep.
-	 */
-	public int getSuperstepNumber() {
-		return this.runtimeContext.getSuperstepNumber();
-	}
-
-	/**
-	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
-	 * all aggregates globally once per superstep and makes them available in the next superstep.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
-	 */
-	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-		return this.runtimeContext.<T>getIterationAggregator(name);
-	}
-
-	/**
-	 * Get the aggregated value that an aggregator computed in the previous iteration.
-	 *
-	 * @param name The name of the aggregator.
-	 * @return The aggregated value of the previous iteration.
-	 */
-	public <T extends Value> T getPreviousIterationAggregate(String name) {
-		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
-	}
-
-	/**
-	 * Gets the broadcast data set registered under the given name. Broadcast data sets
-	 * are available on all parallel instances of a function.
-	 *
-	 * @param name The name under which the broadcast set is registered.
-	 * @return The broadcast data set.
-	 */
-	public <T> Collection<T> getBroadcastSet(String name) {
-		return this.runtimeContext.<T>getBroadcastVariable(name);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal methods
-	// --------------------------------------------------------------------------------------------
-
-	private IterationRuntimeContext runtimeContext;
-
-	public void init(IterationRuntimeContext iterationRuntimeContext) {
-		this.runtimeContext = iterationRuntimeContext;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
deleted file mode 100644
index 0dd39fc..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/CommunityDetection.java
+++ /dev/null
@@ -1,183 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-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.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * Community Detection Algorithm.
- *
- * This implementation expects Long Vertex values and labels. The Vertex values of the input Graph provide the initial label assignments.
- * 
- * Initially, each vertex is assigned a tuple formed of its own initial value along with a score equal to 1.0.
- * The vertices propagate their labels and max scores in iterations, each time adopting the label with the
- * highest score from the list of received messages. The chosen label is afterwards re-scored using the fraction
- * delta/the superstep number. Delta is passed as a parameter and has 0.5 as a default value.
- *
- * The algorithm converges when vertices no longer update their value or when the maximum number of iterations
- * is reached.
- * 
- * @param <K> the Vertex ID type 
- *
- * @see <a href="http://arxiv.org/pdf/0808.2633.pdf">article explaining the algorithm in detail</a>
- */
-public class CommunityDetection<K> implements GraphAlgorithm<K, Long, Double, Graph<K, Long, Double>> {
-
-	private Integer maxIterations;
-
-	private Double delta;
-
-	public CommunityDetection(Integer maxIterations, Double delta) {
-
-		this.maxIterations = maxIterations;
-		this.delta = delta;
-	}
-
-	@Override
-	public Graph<K, Long, Double> run(Graph<K, Long, Double> graph) {
-
-		DataSet<Vertex<K, Tuple2<Long, Double>>> initializedVertices = graph.getVertices()
-				.map(new AddScoreToVertexValuesMapper<K>());
-
-		Graph<K, Tuple2<Long, Double>, Double> graphWithScoredVertices =
-				Graph.fromDataSet(initializedVertices, graph.getEdges(), graph.getContext()).getUndirected();
-
-		return graphWithScoredVertices.runVertexCentricIteration(new VertexLabelUpdater<K>(delta),
-				new LabelMessenger<K>(), maxIterations)
-				.mapVertices(new RemoveScoreFromVertexValuesMapper<K>());
-	}
-
-	@SuppressWarnings("serial")
-	public static final class VertexLabelUpdater<K> extends VertexUpdateFunction<
-		K, Tuple2<Long, Double>, Tuple2<Long, Double>> {
-
-		private Double delta;
-
-		public VertexLabelUpdater(Double delta) {
-			this.delta = delta;
-		}
-
-		@Override
-		public void updateVertex(Vertex<K, Tuple2<Long, Double>> vertex,
-								MessageIterator<Tuple2<Long, Double>> inMessages) throws Exception {
-
-			// we would like these two maps to be ordered
-			Map<Long, Double> receivedLabelsWithScores = new TreeMap<Long, Double>();
-			Map<Long, Double> labelsWithHighestScore = new TreeMap<Long, Double>();
-
-			for (Tuple2<Long, Double> message : inMessages) {
-				// split the message into received label and score
-				Long receivedLabel = message.f0;
-				Double receivedScore = message.f1;
-
-				// if the label was received before
-				if (receivedLabelsWithScores.containsKey(receivedLabel)) {
-					Double newScore = receivedScore + receivedLabelsWithScores.get(receivedLabel);
-					receivedLabelsWithScores.put(receivedLabel, newScore);
-				} else {
-					// first time we see the label
-					receivedLabelsWithScores.put(receivedLabel, receivedScore);
-				}
-
-				// store the labels with the highest scores
-				if (labelsWithHighestScore.containsKey(receivedLabel)) {
-					Double currentScore = labelsWithHighestScore.get(receivedLabel);
-					if (currentScore < receivedScore) {
-						// record the highest score
-						labelsWithHighestScore.put(receivedLabel, receivedScore);
-					}
-				} else {
-					// first time we see this label
-					labelsWithHighestScore.put(receivedLabel, receivedScore);
-				}
-			}
-
-			if(receivedLabelsWithScores.size() > 0) {
-				// find the label with the highest score from the ones received
-				Double maxScore = -Double.MAX_VALUE;
-				Long maxScoreLabel = vertex.getValue().f0;
-				for (Long curLabel : receivedLabelsWithScores.keySet()) {
-
-					if (receivedLabelsWithScores.get(curLabel) > maxScore) {
-						maxScore = receivedLabelsWithScores.get(curLabel);
-						maxScoreLabel = curLabel;
-					}
-				}
-
-				// find the highest score of maxScoreLabel
-				Double highestScore = labelsWithHighestScore.get(maxScoreLabel);
-				// re-score the new label
-				if (maxScoreLabel != vertex.getValue().f0) {
-					highestScore -= delta / getSuperstepNumber();
-				}
-				// else delta = 0
-				// update own label
-				setNewVertexValue(new Tuple2<Long, Double>(maxScoreLabel, highestScore));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class LabelMessenger<K> extends MessagingFunction<K, Tuple2<Long, Double>,
-			Tuple2<Long, Double>, Double> {
-
-		@Override
-		public void sendMessages(Vertex<K, Tuple2<Long, Double>> vertex) throws Exception {
-
-			for(Edge<K, Double> edge : getEdges()) {
-				sendMessageTo(edge.getTarget(), new Tuple2<Long, Double>(vertex.getValue().f0,
-						vertex.getValue().f1 * edge.getValue()));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	@ForwardedFields("f0")
-	public static final class AddScoreToVertexValuesMapper<K> implements MapFunction<
-		Vertex<K, Long>, Vertex<K, Tuple2<Long, Double>>> {
-
-		public Vertex<K, Tuple2<Long, Double>> map(Vertex<K, Long> vertex) {
-			return new Vertex<K, Tuple2<Long, Double>>(
-					vertex.getId(), new Tuple2<Long, Double>(vertex.getValue(), 1.0));
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class RemoveScoreFromVertexValuesMapper<K> implements MapFunction<
-		Vertex<K, Tuple2<Long, Double>>, Long> {
-
-		@Override
-		public Long map(Vertex<K, Tuple2<Long, Double>> vertex) throws Exception {
-			return vertex.getValue().f0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
deleted file mode 100644
index ed853fe..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/ConnectedComponents.java
+++ /dev/null
@@ -1,97 +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.library;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
-import org.apache.flink.types.NullValue;
-
-/**
- * A vertex-centric implementation of the Connected Components algorithm.
- *
- * This implementation assumes that the vertices of the input Graph are initialized with unique, Long component IDs.
- * The vertices propagate their current component ID in iterations, each time adopting a new value from the received neighbor IDs,
- * provided that the value is less than the current minimum.
- *
- * The algorithm converges when vertices no longer update their value or when the maximum number of iterations
- * is reached.
- * 
- * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
- * 
- * @see {@link org.apache.flink.graph.library.GSAConnectedComponents}
- */
-@SuppressWarnings("serial")
-public class ConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {
-
-	private Integer maxIterations;
-
-	public ConnectedComponents(Integer maxIterations) {
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> graph) throws Exception {
-
-		Graph<K, Long, NullValue> undirectedGraph = graph.mapEdges(new NullValueEdgeMapper<K, EV>())
-				.getUndirected();
-
-		// initialize vertex values and run the Vertex Centric Iteration
-		return undirectedGraph.runVertexCentricIteration(
-				new CCUpdater<K>(), new CCMessenger<K>(), maxIterations)
-				.getVertices();
-	}
-
-	/**
-	 * Updates the value of a vertex by picking the minimum neighbor ID out of all the incoming messages.
-	 */
-	public static final class CCUpdater<K> extends VertexUpdateFunction<K, Long, Long> {
-
-		@Override
-		public void updateVertex(Vertex<K, Long> vertex, MessageIterator<Long> messages) throws Exception {
-			long min = Long.MAX_VALUE;
-
-			for (long msg : messages) {
-				min = Math.min(min, msg);
-			}
-
-			// update vertex value, if new minimum
-			if (min < vertex.getValue()) {
-				setNewVertexValue(min);
-			}
-		}
-	}
-
-	/**
-	 * Distributes the minimum ID associated with a given vertex among all the target vertices.
-	 */
-	public static final class CCMessenger<K> extends MessagingFunction<K, Long, Long, NullValue> {
-
-		@Override
-		public void sendMessages(Vertex<K, Long> vertex) throws Exception {
-			// send current minimum to neighbors
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
deleted file mode 100755
index 77bc2cf..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java
+++ /dev/null
@@ -1,88 +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.library;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.gsa.ApplyFunction;
-import org.apache.flink.graph.gsa.GatherFunction;
-import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.graph.gsa.Neighbor;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
-import org.apache.flink.types.NullValue;
-
-/**
- * This is an implementation of the Connected Components algorithm, using a gather-sum-apply iteration.
- * This implementation assumes that the vertices of the input Graph are initialized with unique, Long component IDs.
- * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
- * 
- * @see {@link org.apache.flink.graph.library.ConnectedComponents}
- */
-public class GSAConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {
-
-	private Integer maxIterations;
-
-	public GSAConnectedComponents(Integer maxIterations) {
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> graph) throws Exception {
-
-		Graph<K, Long, NullValue> undirectedGraph = graph.mapEdges(new NullValueEdgeMapper<K, EV>())
-				.getUndirected();
-
-		// initialize vertex values and run the Vertex Centric Iteration
-		return undirectedGraph.runGatherSumApplyIteration(
-				new GatherNeighborIds(), new SelectMinId(), new UpdateComponentId<K>(),
-				maxIterations).getVertices();
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Connected Components UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	private static final class GatherNeighborIds extends GatherFunction<Long, NullValue, Long> {
-
-		public Long gather(Neighbor<Long, NullValue> neighbor) {
-			return neighbor.getNeighborValue();
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class SelectMinId extends SumFunction<Long, NullValue, Long> {
-
-		public Long sum(Long newValue, Long currentValue) {
-			return Math.min(newValue, currentValue);
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class UpdateComponentId<K> extends ApplyFunction<K, Long, Long> {
-
-		public void apply(Long summedValue, Long origValue) {
-			if (summedValue < origValue) {
-				setResult(summedValue);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
deleted file mode 100644
index df3e89a..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
+++ /dev/null
@@ -1,135 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-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;
-
-/**
- * This is an implementation of a simple PageRank algorithm, using a gather-sum-apply iteration.
- * The user can define the damping factor and the maximum number of iterations.
- * If the number of vertices of the input graph is known, it should be provided as a parameter
- * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices.
- * 
- * The implementation assumes that each page has at least one incoming and one outgoing link.
- */
-public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
-
-	private double beta;
-	private int maxIterations;
-	private long numberOfVertices;
-
-	/**
-	 * @param beta the damping factor
-	 * @param maxIterations the maximum number of iterations
-	 */
-	public GSAPageRank(double beta, int maxIterations) {
-		this.beta = beta;
-		this.maxIterations = maxIterations;
-	}
-
-	public GSAPageRank(double beta, long numVertices, int maxIterations) {
-		this.beta = beta;
-		this.numberOfVertices = numVertices;
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> network) throws Exception {
-
-		if (numberOfVertices == 0) {
-			numberOfVertices = network.numberOfVertices();
-		}
-
-		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
-
-		Graph<K, Double, Double> networkWithWeights = network
-				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
-
-		return networkWithWeights.runGatherSumApplyIteration(new GatherRanks(numberOfVertices), new SumRanks(),
-				new UpdateRanks<K>(beta, numberOfVertices), maxIterations)
-				.getVertices();
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Page Rank UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	private static final class GatherRanks extends GatherFunction<Double, Double, Double> {
-
-		long numberOfVertices;
-
-		public GatherRanks(long numberOfVertices) {
-			this.numberOfVertices = numberOfVertices;
-		}
-
-		@Override
-		public Double gather(Neighbor<Double, Double> neighbor) {
-			double neighborRank = neighbor.getNeighborValue();
-
-			if(getSuperstepNumber() == 1) {
-				neighborRank = 1.0 / numberOfVertices;
-			}
-
-			return neighborRank * neighbor.getEdgeValue();
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class SumRanks extends SumFunction<Double, Double, Double> {
-
-		@Override
-		public Double sum(Double newValue, Double currentValue) {
-			return newValue + currentValue;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class UpdateRanks<K> extends ApplyFunction<K, Double, Double> {
-
-		private final double beta;
-		private final long numVertices;
-
-		public UpdateRanks(double beta, long numberOfVertices) {
-			this.beta = beta;
-			this.numVertices = numberOfVertices;
-		}
-
-		@Override
-		public void apply(Double rankSum, Double currentValue) {
-			setResult((1-beta)/numVertices + beta * rankSum);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
-		public Double map(Tuple2<Double, Long> value) {
-			return value.f0 / value.f1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
deleted file mode 100755
index 5a76072..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSASingleSourceShortestPaths.java
+++ /dev/null
@@ -1,101 +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.library;
-
-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.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.gsa.ApplyFunction;
-import org.apache.flink.graph.gsa.GatherFunction;
-import org.apache.flink.graph.gsa.SumFunction;
-import org.apache.flink.graph.gsa.Neighbor;
-
-/**
- * This is an implementation of the Single Source Shortest Paths algorithm, using a gather-sum-apply iteration
- */
-public class GSASingleSourceShortestPaths<K> implements
-	GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
-
-	private final K srcVertexId;
-	private final Integer maxIterations;
-
-	public GSASingleSourceShortestPaths(K srcVertexId, Integer maxIterations) {
-		this.srcVertexId = srcVertexId;
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> input) {
-
-		return input.mapVertices(new InitVerticesMapper<K>(srcVertexId))
-				.runGatherSumApplyIteration(new CalculateDistances(), new ChooseMinDistance(),
-						new UpdateDistance<K>(), maxIterations)
-						.getVertices();
-	}
-
-	@SuppressWarnings("serial")
-	public static final class InitVerticesMapper<K>	implements MapFunction<Vertex<K, Double>, Double> {
-
-		private K srcVertexId;
-
-		public InitVerticesMapper(K srcId) {
-			this.srcVertexId = srcId;
-		}
-
-		public Double map(Vertex<K, Double> value) {
-			if (value.f0.equals(srcVertexId)) {
-				return 0.0;
-			} else {
-				return Double.MAX_VALUE;
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Single Source Shortest Path UDFs
-	// --------------------------------------------------------------------------------------------
-
-	@SuppressWarnings("serial")
-	private static final class CalculateDistances extends GatherFunction<Double, Double, Double> {
-
-		public Double gather(Neighbor<Double, Double> neighbor) {
-			return neighbor.getNeighborValue() + neighbor.getEdgeValue();
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class ChooseMinDistance extends SumFunction<Double, Double, Double> {
-
-		public Double sum(Double newValue, Double currentValue) {
-			return Math.min(newValue, currentValue);
-		}
-	};
-
-	@SuppressWarnings("serial")
-	private static final class UpdateDistance<K> extends ApplyFunction<K, Double, Double> {
-
-		public void apply(Double newDistance, Double oldDistance) {
-			if (newDistance < oldDistance) {
-				setResult(newDistance);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
deleted file mode 100644
index 76d170d..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSATriangleCount.java
+++ /dev/null
@@ -1,190 +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.library;
-
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-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.GraphAlgorithm;
-import org.apache.flink.graph.ReduceNeighborsFunction;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.Triplet;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.types.NullValue;
-
-import java.util.TreeMap;
-
-/**
- * Triangle Count Algorithm.
- *
- * This algorithm operates in three phases. First, vertices select neighbors with id greater than theirs
- * and send messages to them. Each received message is then propagated to neighbors with higher id.
- * Finally, if a node encounters the target id in the list of received messages, it increments the number
- * of triangles found.
- *
- * This implementation is non - iterative.
- *
- * The algorithm takes an undirected, unweighted graph as input and outputs a DataSet
- * which contains a single integer representing the number of triangles.
- */
-public class GSATriangleCount<K extends Comparable<K>, VV, EV> implements
-		GraphAlgorithm<K, VV, EV, DataSet<Integer>> {
-
-	@SuppressWarnings("serial")
-	@Override
-	public DataSet<Integer> run(Graph<K, VV, EV> input) throws Exception {
-
-		ExecutionEnvironment env = input.getContext();
-
-		// order the edges so that src is always higher than trg
-		DataSet<Edge<K, NullValue>> edges = input.getEdges().map(new OrderEdges<K, EV>()).distinct();
-
-		Graph<K, TreeMap<K, Integer>, NullValue> graph = Graph.fromDataSet(edges,
-				new VertexInitializer<K>(), env);
-
-		// select neighbors with ids higher than the current vertex id
-		// Gather: a no-op in this case
-		// Sum: create the set of neighbors
-		DataSet<Tuple2<K, TreeMap<K, Integer>>> higherIdNeighbors =
-				graph.reduceOnNeighbors(new GatherHigherIdNeighbors<K>(), EdgeDirection.IN);
-
-		Graph<K, TreeMap<K, Integer>, NullValue> graphWithReinitializedVertexValues =
-				graph.mapVertices(new VertexInitializerEmptyTreeMap<K>());
-
-		// Apply: attach the computed values to the vertices
-		// joinWithVertices to update the node values
-		DataSet<Vertex<K, TreeMap<K, Integer>>> verticesWithHigherIdNeighbors =
-				graphWithReinitializedVertexValues.joinWithVertices(higherIdNeighbors, new AttachValues<K>()).getVertices();
-
-		Graph<K, TreeMap<K,Integer>, NullValue> graphWithNeighbors = Graph.fromDataSet(verticesWithHigherIdNeighbors,
-				edges, env);
-
-		// propagate each received value to neighbors with higher id
-		// Gather: a no-op in this case
-		// Sum: propagate values
-		DataSet<Tuple2<K, TreeMap<K, Integer>>> propagatedValues = graphWithNeighbors
-				.reduceOnNeighbors(new GatherHigherIdNeighbors<K>(), EdgeDirection.IN);
-
-		// Apply: attach propagated values to vertices
-		DataSet<Vertex<K, TreeMap<K, Integer>>> verticesWithPropagatedValues =
-				graphWithReinitializedVertexValues.joinWithVertices(propagatedValues, new AttachValues<K>()).getVertices();
-
-		Graph<K, TreeMap<K, Integer>, NullValue> graphWithPropagatedNeighbors =
-				Graph.fromDataSet(verticesWithPropagatedValues, graphWithNeighbors.getEdges(), env);
-
-		// Scatter: compute the number of triangles
-		DataSet<Integer> numberOfTriangles = graphWithPropagatedNeighbors.getTriplets()
-				.map(new ComputeTriangles<K>()).reduce(new ReduceFunction<Integer>() {
-
-					@Override
-					public Integer reduce(Integer first, Integer second) throws Exception {
-						return first + second;
-					}
-				});
-
-		return numberOfTriangles;
-	}
-
-	@SuppressWarnings("serial")
-	private static final class OrderEdges<K extends Comparable<K>, EV> implements
-		MapFunction<Edge<K, EV>, Edge<K, NullValue>> {
-
-		@Override
-		public Edge<K, NullValue> map(Edge<K, EV> edge) throws Exception {
-			if (edge.getSource().compareTo(edge.getTarget()) < 0) {
-				return new Edge<K, NullValue>(edge.getTarget(), edge.getSource(), NullValue.getInstance());
-			} else {
-				return new Edge<K, NullValue>(edge.getSource(), edge.getTarget(), NullValue.getInstance());
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class VertexInitializer<K> implements MapFunction<K, TreeMap<K, Integer>> {
-
-		@Override
-		public TreeMap<K, Integer> map(K value) throws Exception {
-			TreeMap<K, Integer> neighbors = new TreeMap<K, Integer>();
-			neighbors.put(value, 1);
-
-			return neighbors;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class VertexInitializerEmptyTreeMap<K> implements
-			MapFunction<Vertex<K, TreeMap<K, Integer>>, TreeMap<K, Integer>> {
-
-		@Override
-		public TreeMap<K, Integer> map(Vertex<K, TreeMap<K, Integer>> vertex) throws Exception {
-			return new TreeMap<K, Integer>();
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class AttachValues<K> implements MapFunction<Tuple2<TreeMap<K, Integer>,
-			TreeMap<K, Integer>>, TreeMap<K, Integer>> {
-
-		@Override
-		public TreeMap<K, Integer> map(Tuple2<TreeMap<K, Integer>, TreeMap<K, Integer>> tuple2) throws Exception {
-			return tuple2.f1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class GatherHigherIdNeighbors<K> implements
-		ReduceNeighborsFunction<TreeMap<K,Integer>> {
-
-		@Override
-		public TreeMap<K, Integer> reduceNeighbors(TreeMap<K,Integer> first, TreeMap<K,Integer> second) {
-			for (K key : second.keySet()) {
-				Integer value = first.get(key);
-				if (value != null) {
-					first.put(key, value + second.get(key));
-				} else {
-					first.put(key, second.get(key));
-				}
-			}
-			return first;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class ComputeTriangles<K> implements MapFunction<Triplet<K, TreeMap<K, Integer>, NullValue>,
-			Integer> {
-
-		@Override
-		public Integer map(Triplet<K, TreeMap<K, Integer>, NullValue> triplet) throws Exception {
-
-			Vertex<K, TreeMap<K, Integer>> srcVertex = triplet.getSrcVertex();
-			Vertex<K, TreeMap<K, Integer>> trgVertex = triplet.getTrgVertex();
-			int triangles = 0;
-
-			if(trgVertex.getValue().get(srcVertex.getId()) != null) {
-				triangles = trgVertex.getValue().get(srcVertex.getId());
-			}
-			return triangles;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
deleted file mode 100644
index 82dfee7..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/LabelPropagation.java
+++ /dev/null
@@ -1,116 +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.library;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.graph.Graph;
-import org.apache.flink.graph.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-import org.apache.flink.graph.utils.NullValueEdgeMapper;
-import org.apache.flink.types.NullValue;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-/**
- * An implementation of the label propagation algorithm. The iterative algorithm
- * detects communities by propagating labels. In each iteration, a vertex adopts
- * the label that is most frequent among its neighbors' labels. Labels are
- * represented by Longs and we assume a total ordering among them, in order to
- * break ties. The algorithm converges when no vertex changes its value or the
- * maximum number of iterations have been reached. Note that different
- * initializations might lead to different results.
- * 
- */
-@SuppressWarnings("serial")
-
-public class LabelPropagation<K extends Comparable<K>, EV> implements GraphAlgorithm<K, Long, EV,
-	DataSet<Vertex<K, Long>>> {
-
-	private final int maxIterations;
-
-	public LabelPropagation(int maxIterations) {
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Long>> run(Graph<K, Long, EV> input) {
-
-		// iteratively adopt the most frequent label among the neighbors
-		// of each vertex
-		return input.mapEdges(new NullValueEdgeMapper<K, EV>()).runVertexCentricIteration(new UpdateVertexLabel<K>(), new SendNewLabelToNeighbors<K>(),
-				maxIterations).getVertices();
-	}
-
-	/**
-	 * Function that updates the value of a vertex by adopting the most frequent
-	 * label among its in-neighbors
-	 */
-	public static final class UpdateVertexLabel<K> extends VertexUpdateFunction<K, Long, Long> {
-
-		public void updateVertex(Vertex<K, Long> vertex,
-				MessageIterator<Long> inMessages) {
-			Map<Long, Long> labelsWithFrequencies = new HashMap<Long, Long>();
-
-			long maxFrequency = 1;
-			long mostFrequentLabel = vertex.getValue();
-
-			// store the labels with their frequencies
-			for (Long msg : inMessages) {
-				if (labelsWithFrequencies.containsKey(msg)) {
-					long currentFreq = labelsWithFrequencies.get(msg);
-					labelsWithFrequencies.put(msg, currentFreq + 1);
-				} else {
-					labelsWithFrequencies.put(msg, 1L);
-				}
-			}
-			// select the most frequent label: if two or more labels have the
-			// same frequency,
-			// the node adopts the label with the highest value
-			for (Entry<Long, Long> entry : labelsWithFrequencies.entrySet()) {
-				if (entry.getValue() == maxFrequency) {
-					// check the label value to break ties
-					if (entry.getKey() > mostFrequentLabel) {
-						mostFrequentLabel = entry.getKey();
-					}
-				} else if (entry.getValue() > maxFrequency) {
-					maxFrequency = entry.getValue();
-					mostFrequentLabel = entry.getKey();
-				}
-			}
-
-			// set the new vertex value
-			setNewVertexValue(mostFrequentLabel);
-		}
-	}
-
-	/**
-	 * Sends the vertex label to all out-neighbors
-	 */
-	public static final class SendNewLabelToNeighbors<K> extends MessagingFunction<K, Long, Long, NullValue> {
-
-		public void sendMessages(Vertex<K, Long> vertex) {
-			sendMessageToAllNeighbors(vertex.getValue());
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
deleted file mode 100644
index 8193dba..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
+++ /dev/null
@@ -1,146 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-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.GraphAlgorithm;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.graph.spargel.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-
-/**
- * This is an implementation of a simple PageRank algorithm, using a vertex-centric iteration.
- * The user can define the damping factor and the maximum number of iterations.
- * If the number of vertices of the input graph is known, it should be provided as a parameter
- * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices.
- * 
- * The implementation assumes that each page has at least one incoming and one outgoing link.
- */
-public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
-
-	private double beta;
-	private int maxIterations;
-	private long numberOfVertices;
-
-	/**
-	 * @param beta the damping factor
-	 * @param maxIterations the maximum number of iterations
-	 */
-	public PageRank(double beta, int maxIterations) {
-		this.beta = beta;
-		this.maxIterations = maxIterations;
-		this.numberOfVertices = 0;
-	}
-
-	/**
-	 * @param beta the damping factor
-	 * @param maxIterations the maximum number of iterations
-	 * @param numVertices the number of vertices in the input
-	 */
-	public PageRank(double beta, long numVertices, int maxIterations) {
-		this.beta = beta;
-		this.maxIterations = maxIterations;
-		this.numberOfVertices = numVertices;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> network) throws Exception {
-
-		if (numberOfVertices == 0) {
-			numberOfVertices = network.numberOfVertices();
-		}
-
-		DataSet<Tuple2<K, Long>> vertexOutDegrees = network.outDegrees();
-
-		Graph<K, Double, Double> networkWithWeights = network
-				.joinWithEdgesOnSource(vertexOutDegrees, new InitWeightsMapper());
-
-		return networkWithWeights.runVertexCentricIteration(new VertexRankUpdater<K>(beta, numberOfVertices),
-				new RankMessenger<K>(numberOfVertices), maxIterations)
-				.getVertices();
-	}
-
-	/**
-	 * Function that updates the rank of a vertex by summing up the partial
-	 * ranks from all incoming messages and then applying the dampening formula.
-	 */
-	@SuppressWarnings("serial")
-	public static final class VertexRankUpdater<K> extends VertexUpdateFunction<K, Double, Double> {
-
-		private final double beta;
-		private final long numVertices;
-		
-		public VertexRankUpdater(double beta, long numberOfVertices) {
-			this.beta = beta;
-			this.numVertices = numberOfVertices;
-		}
-
-		@Override
-		public void updateVertex(Vertex<K, Double> vertex, MessageIterator<Double> inMessages) {
-			double rankSum = 0.0;
-			for (double msg : inMessages) {
-				rankSum += msg;
-			}
-
-			// apply the dampening factor / random jump
-			double newRank = (beta * rankSum) + (1 - beta) / numVertices;
-			setNewVertexValue(newRank);
-		}
-	}
-
-	/**
-	 * Distributes the rank of a vertex among all target vertices according to
-	 * the transition probability, which is associated with an edge as the edge
-	 * value.
-	 */
-	@SuppressWarnings("serial")
-	public static final class RankMessenger<K> extends MessagingFunction<K, Double, Double, Double> {
-
-		private final long numVertices;
-
-		public RankMessenger(long numberOfVertices) {
-			this.numVertices = numberOfVertices;
-		}
-
-		@Override
-		public void sendMessages(Vertex<K, Double> vertex) {
-			if (getSuperstepNumber() == 1) {
-				// initialize vertex ranks
-				vertex.setValue(new Double(1.0 / numVertices));
-			}
-
-			for (Edge<K, Double> edge : getEdges()) {
-				sendMessageTo(edge.getTarget(), vertex.getValue() * edge.getValue());
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	private static final class InitWeightsMapper implements MapFunction<Tuple2<Double, Long>, Double> {
-		public Double map(Tuple2<Double, Long> value) {
-			return value.f0 / value.f1;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
deleted file mode 100644
index 60c4c17..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/SingleSourceShortestPaths.java
+++ /dev/null
@@ -1,112 +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.library;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-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.MessageIterator;
-import org.apache.flink.graph.spargel.MessagingFunction;
-import org.apache.flink.graph.spargel.VertexUpdateFunction;
-
-/**
- * This is an implementation of the Single-Source-Shortest Paths algorithm, using a vertex-centric iteration.
- */
-@SuppressWarnings("serial")
-public class SingleSourceShortestPaths<K> implements GraphAlgorithm<K, Double, Double, DataSet<Vertex<K, Double>>> {
-
-	private final K srcVertexId;
-	private final Integer maxIterations;
-
-	public SingleSourceShortestPaths(K srcVertexId, Integer maxIterations) {
-		this.srcVertexId = srcVertexId;
-		this.maxIterations = maxIterations;
-	}
-
-	@Override
-	public DataSet<Vertex<K, Double>> run(Graph<K, Double, Double> input) {
-
-		return input.mapVertices(new InitVerticesMapper<K>(srcVertexId))
-				.runVertexCentricIteration(new VertexDistanceUpdater<K>(), new MinDistanceMessenger<K>(),
-				maxIterations).getVertices();
-	}
-
-	public static final class InitVerticesMapper<K>	implements MapFunction<Vertex<K, Double>, Double> {
-
-		private K srcVertexId;
-
-		public InitVerticesMapper(K srcId) {
-			this.srcVertexId = srcId;
-		}
-
-		public Double map(Vertex<K, Double> value) {
-			if (value.f0.equals(srcVertexId)) {
-				return 0.0;
-			} else {
-				return Double.MAX_VALUE;
-			}
-		}
-	}
-
-	/**
-	 * Function that updates the value of a vertex by picking the minimum
-	 * distance from all incoming messages.
-	 * 
-	 * @param <K>
-	 */
-	public static final class VertexDistanceUpdater<K> extends VertexUpdateFunction<K, Double, Double> {
-
-		@Override
-		public void updateVertex(Vertex<K, Double> vertex,
-				MessageIterator<Double> inMessages) {
-
-			Double minDistance = Double.MAX_VALUE;
-
-			for (double msg : inMessages) {
-				if (msg < minDistance) {
-					minDistance = msg;
-				}
-			}
-
-			if (vertex.getValue() > minDistance) {
-				setNewVertexValue(minDistance);
-			}
-		}
-	}
-
-	/**
-	 * Distributes the minimum distance associated with a given vertex among all
-	 * the target vertices summed up with the edge's value.
-	 * 
-	 * @param <K>
-	 */
-	public static final class MinDistanceMessenger<K> extends MessagingFunction<K, Double, Double, Double> {
-
-		@Override
-		public void sendMessages(Vertex<K, Double> vertex)
-				throws Exception {
-			for (Edge<K, Double> edge : getEdges()) {
-				sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue());
-			}
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
deleted file mode 100644
index d6fdc8a..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessageIterator.java
+++ /dev/null
@@ -1,58 +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.spargel;
-
-import java.util.Iterator;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
-/**
- * An iterator that returns messages. The iterator is {@link java.lang.Iterable} at the same time to support
- * the <i>foreach</i> syntax.
- */
-public final class MessageIterator<Message> implements Iterator<Message>, Iterable<Message>, java.io.Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private transient Iterator<Tuple2<?, Message>> source;
-	
-	
-	final void setSource(Iterator<Tuple2<?, Message>> source) {
-		this.source = source;
-	}
-	
-	@Override
-	public final boolean hasNext() {
-		return this.source.hasNext();
-	}
-	
-	@Override
-	public final Message next() {
-		return this.source.next().f1;
-	}
-
-	@Override
-	public final void remove() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public Iterator<Message> iterator() {
-		return this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java b/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
deleted file mode 100644
index 4245c24..0000000
--- a/flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
+++ /dev/null
@@ -1,303 +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.spargel;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Iterator;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-import org.apache.flink.api.common.functions.IterationRuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.graph.Edge;
-import org.apache.flink.graph.EdgeDirection;
-import org.apache.flink.graph.Vertex;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.Collector;
-
-/**
- * The base class for functions that produce messages between vertices as a part of a {@link VertexCentricIteration}.
- * 
- * @param <K> The type of the vertex key (the vertex identifier).
- * @param <VV> The type of the vertex value (the state of the vertex).
- * @param <Message> The type of the message sent between vertices along the edges.
- * @param <EV> The type of the values that are associated with the edges.
- */
-public abstract class MessagingFunction<K, VV, Message, EV> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	// --------------------------------------------------------------------------------------------
-	//  Attributes that allow vertices to access their in/out degrees and the total number of vertices
-	//  inside an iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private long numberOfVertices = -1L;
-
-	/**
-	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
-	 * option has been set; -1 otherwise.
-	 */
-	public long getNumberOfVertices() {
-		return numberOfVertices;
-	}
-
-	void setNumberOfVertices(long numberOfVertices) {
-		this.numberOfVertices = numberOfVertices;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Attribute that allows the user to choose the neighborhood type(in/out/all) on which to run
-	//  the vertex centric iteration.
-	// --------------------------------------------------------------------------------------------
-
-	private EdgeDirection direction;
-
-	/**
-	 * Retrieves the edge direction in which messages are propagated in the vertex-centric iteration.
-	 * @return the messaging {@link EdgeDirection}
-	 */
-	public EdgeDirection getDirection() {
-		return direction;
-	}
-
-	void setDirection(EdgeDirection direction) {
-		this.direction = direction;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Public API Methods
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * This method is invoked once per superstep for each vertex that was changed in that superstep.
-	 * It needs to produce the messages that will be received by vertices in the next superstep.
-	 * 
-	 * @param vertex The vertex that was changed.
-	 * 
-	 * @throws Exception The computation may throw exceptions, which causes the superstep to fail.
-	 */
-	public abstract void sendMessages(Vertex<K, VV> vertex) throws Exception;
-	
-	/**
-	 * This method is executed one per superstep before the vertex update function is invoked for each vertex.
-	 * 
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
-	 */
-	public void preSuperstep() throws Exception {}
-	
-	/**
-	 * This method is executed one per superstep after the vertex update function has been invoked for each vertex.
-	 * 
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
-	 */
-	public void postSuperstep() throws Exception {}
-	
-	
-	/**
-	 * Gets an {@link java.lang.Iterable} with all edges. This method is mutually exclusive with
-	 * {@link #sendMessageToAllNeighbors(Object)} and may be called only once.
-	 * 
-	 * @return An iterator with all outgoing edges.
-	 */
-	@SuppressWarnings("unchecked")
-	public Iterable<Edge<K, EV>> getEdges() {
-		if (edgesUsed) {
-			throw new IllegalStateException("Can use either 'getEdges()' or 'sendMessageToAllTargets()' exactly once.");
-		}
-		edgesUsed = true;
-		this.edgeIterator.set((Iterator<Edge<K, EV>>) edges);
-		return this.edgeIterator;
-	}
-
-	/**
-	 * Sends the given message to all vertices that are targets of an outgoing edge of the changed vertex.
-	 * This method is mutually exclusive to the method {@link #getEdges()} and may be called only once.
-	 * 
-	 * @param m The message to send.
-	 */
-	public void sendMessageToAllNeighbors(Message m) {
-		if (edgesUsed) {
-			throw new IllegalStateException("Can use either 'getEdges()' or 'sendMessageToAllTargets()' exactly once.");
-		}
-		
-		edgesUsed = true;
-		
-		outValue.f1 = m;
-		
-		while (edges.hasNext()) {
-			Tuple next = (Tuple) edges.next();
-			K k = next.getField(1);
-			outValue.f0 = k;
-			out.collect(outValue);
-		}
-	}
-	
-	/**
-	 * Sends the given message to the vertex identified by the given key. If the target vertex does not exist,
-	 * the next superstep will cause an exception due to a non-deliverable message.
-	 * 
-	 * @param target The key (id) of the target vertex to message.
-	 * @param m The message.
-	 */
-	public void sendMessageTo(K target, Message m) {
-		outValue.f0 = target;
-		outValue.f1 = m;
-		out.collect(outValue);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Gets the number of the superstep, starting at <tt>1</tt>.
-	 * 
-	 * @return The number of the current superstep.
-	 */
-	public int getSuperstepNumber() {
-		return this.runtimeContext.getSuperstepNumber();
-	}
-	
-	/**
-	 * Gets the iteration aggregator registered under the given name. The iteration aggregator combines
-	 * all aggregates globally once per superstep and makes them available in the next superstep.
-	 * 
-	 * @param name The name of the aggregator.
-	 * @return The aggregator registered under this name, or null, if no aggregator was registered.
-	 */
-	public <T extends Aggregator<?>> T getIterationAggregator(String name) {
-		return this.runtimeContext.<T>getIterationAggregator(name);
-	}
-	
-	/**
-	 * Get the aggregated value that an aggregator computed in the previous iteration.
-	 * 
-	 * @param name The name of the aggregator.
-	 * @return The aggregated value of the previous iteration.
-	 */
-	public <T extends Value> T getPreviousIterationAggregate(String name) {
-		return this.runtimeContext.<T>getPreviousIterationAggregate(name);
-	}
-	
-	/**
-	 * Gets the broadcast data set registered under the given name. Broadcast data sets
-	 * are available on all parallel instances of a function. They can be registered via
-	 * {@link org.apache.flink.graph.spargel.VertexCentricConfiguration#addBroadcastSetForMessagingFunction(String, org.apache.flink.api.java.DataSet)}.
-	 * 
-	 * @param name The name under which the broadcast set is registered.
-	 * @return The broadcast data set.
-	 */
-	public <T> Collection<T> getBroadcastSet(String name) {
-		return this.runtimeContext.<T>getBroadcastVariable(name);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  internal methods and state
-	// --------------------------------------------------------------------------------------------
-	
-	private Tuple2<K, Message> outValue;
-	
-	private IterationRuntimeContext runtimeContext;
-	
-	private Iterator<?> edges;
-	
-	private Collector<Tuple2<K, Message>> out;
-	
-	private EdgesIterator<K, EV> edgeIterator;
-	
-	private boolean edgesUsed;
-
-	private long inDegree = -1;
-
-	private long outDegree = -1;
-	
-	void init(IterationRuntimeContext context) {
-		this.runtimeContext = context;
-		this.outValue = new Tuple2<K, Message>();
-		this.edgeIterator = new EdgesIterator<K, EV>();
-	}
-	
-	void set(Iterator<?> edges, Collector<Tuple2<K, Message>> out) {
-		this.edges = edges;
-		this.out = out;
-		this.edgesUsed = false;
-	}
-	
-	private static final class EdgesIterator<K, EV> 
-		implements Iterator<Edge<K, EV>>, Iterable<Edge<K, EV>>
-	{
-		private Iterator<Edge<K, EV>> input;
-		
-		private Edge<K, EV> edge = new Edge<K, EV>();
-		
-		void set(Iterator<Edge<K, EV>> input) {
-			this.input = input;
-		}
-		
-		@Override
-		public boolean hasNext() {
-			return input.hasNext();
-		}
-
-		@Override
-		public Edge<K, EV> next() {
-			Edge<K, EV> next = input.next();
-			edge.setSource(next.f0);
-			edge.setTarget(next.f1);
-			edge.setValue(next.f2);
-			return edge;
-		}
-
-		@Override
-		public void remove() {
-			throw new UnsupportedOperationException();
-		}
-		@Override
-		public Iterator<Edge<K, EV>> iterator() {
-			return this;
-		}
-	}
-
-	/**
-	 * Retrieves the vertex in-degree (number of in-coming edges).
-	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
-	 */
-	public long getInDegree() {
-		return inDegree;
-	}
-
-	void setInDegree(long inDegree) {
-		this.inDegree = inDegree;
-	}
-
-	/**
-	 * Retrieve the vertex out-degree (number of out-going edges).
-	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
-	 */
-	public long getOutDegree() {
-		return outDegree;
-	}
-
-	void setOutDegree(long outDegree) {
-		this.outDegree = outDegree;
-	}
-}


[14/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
new file mode 100644
index 0000000..ffc9da9
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/GraphOperationsITCase.java
@@ -0,0 +1,378 @@
+/*
+ * 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.operations;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+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.Triplet;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class GraphOperationsITCase extends MultipleProgramsTestBase {
+
+	public GraphOperationsITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testUndirected() throws Exception {
+		/*
+		 * Test getUndirected()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Edge<Long,Long>> data = graph.getUndirected().getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+        
+		expectedResult = "1,2,12\n" + "2,1,12\n" +
+					"1,3,13\n" + "3,1,13\n" +
+					"2,3,23\n" + "3,2,23\n" +
+					"3,4,34\n" + "4,3,34\n" +
+					"3,5,35\n" + "5,3,35\n" +
+					"4,5,45\n" + "5,4,45\n" +
+					"5,1,51\n" + "1,5,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testReverse() throws Exception {
+		/*
+		 * Test reverse()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Edge<Long,Long>> data = graph.reverse().getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+        
+		expectedResult = "2,1,12\n" +
+					"3,1,13\n" +
+					"3,2,23\n" +
+					"4,3,34\n" +
+					"5,3,35\n" +
+					"5,4,45\n" +
+					"1,5,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	@Test
+	public void testSubGraph() throws Exception {
+		/*
+		 * Test subgraph:
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long,Long>> data= graph.subgraph(new FilterFunction<Vertex<Long, Long>>() {
+						   public boolean filter(Vertex<Long, Long> vertex) throws Exception {
+							   return (vertex.getValue() > 2);
+						   }
+					   },
+				new FilterFunction<Edge<Long, Long>>() {
+					public boolean filter(Edge<Long, Long> edge) throws Exception {
+						return (edge.getValue() > 34);
+					}
+				}).getEdges();
+
+        List<Edge<Long, Long>> result= data.collect();
+        
+		expectedResult = "3,5,35\n" +
+					"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	@Test
+	public void testFilterVertices() throws Exception {
+		/*
+		 * Test filterOnVertices:
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long,Long>> data = graph.filterOnVertices(new FilterFunction<Vertex<Long, Long>>() {
+			public boolean filter(Vertex<Long, Long> vertex) throws Exception {
+				return (vertex.getValue() > 2);
+			}
+		}).getEdges();
+
+        List<Edge<Long, Long>> result= data.collect();
+		
+		expectedResult =  "3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	@Test
+	public void testFilterEdges() throws Exception {
+		/*
+		 * Test filterOnEdges:
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long,Long>> data = graph.filterOnEdges(new FilterFunction<Edge<Long, Long>>() {
+			public boolean filter(Edge<Long, Long> edge) throws Exception {
+				return (edge.getValue() > 34);
+			}
+		}).getEdges();
+
+        List<Edge<Long, Long>> result = data.collect();
+        
+		expectedResult = "3,5,35\n" +
+					"4,5,45\n" +
+					"5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testNumberOfVertices() throws Exception {
+		/*
+		 * Test numberOfVertices()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		DataSet<Long> data = env.fromElements(graph.numberOfVertices());
+
+        List<Long> result= data.collect();
+        
+		expectedResult = "5";
+		
+		compareResultAsText(result, expectedResult);
+	}
+
+	@Test
+	public void testNumberOfEdges() throws Exception {
+		/*
+		 * Test numberOfEdges()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		DataSet<Long> data = env.fromElements(graph.numberOfEdges());
+
+        List<Long> result= data.collect();
+        
+		expectedResult = "7";
+		
+		compareResultAsText(result, expectedResult);
+	}
+
+	@Test
+	public void testVertexIds() throws Exception {
+		/*
+		 * Test getVertexIds()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Long> data = graph.getVertexIds();
+        List<Long> result= data.collect();
+        
+		expectedResult = "1\n2\n3\n4\n5\n";
+		
+		compareResultAsText(result, expectedResult);
+	}
+
+	@Test
+	public void testEdgesIds() throws Exception {
+		/*
+		 * Test getEdgeIds()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Tuple2<Long,Long>> data = graph.getEdgeIds();
+        List<Tuple2<Long, Long>> result= data.collect();
+        
+		expectedResult = "1,2\n" + "1,3\n" +
+				"2,3\n" + "3,4\n" +
+				"3,5\n" + "4,5\n" +
+				"5,1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testUnion() throws Exception {
+		/*
+		 * Test union()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		List<Vertex<Long, Long>> vertices = new ArrayList<Vertex<Long, Long>>();
+		List<Edge<Long, Long>> edges = new ArrayList<Edge<Long, Long>>();
+
+		vertices.add(new Vertex<Long, Long>(6L, 6L));
+		edges.add(new Edge<Long, Long>(6L, 1L, 61L));
+
+		graph = graph.union(Graph.fromCollection(vertices, edges, env));
+
+        DataSet<Edge<Long,Long>> data = graph.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+		expectedResult = "1,2,12\n" +
+					"1,3,13\n" +
+					"2,3,23\n" +
+					"3,4,34\n" +
+					"3,5,35\n" +
+					"4,5,45\n" +
+					"5,1,51\n" +
+					"6,1,61\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testDifference() throws Exception {
+		/*Test  difference() method  by checking    the output  for getEdges()   on  the resultant   graph
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(TestGraphUtils.getLongLongVertexDataDifference(env),
+				TestGraphUtils.getLongLongEdgeDataDifference(env), env);
+
+		graph = graph.difference(graph2);
+
+		List<Edge<Long, Long>> result = graph.getEdges().collect();
+
+		expectedResult = "4,5,45\n";
+		compareResultAsTuples(result, expectedResult);
+	}
+
+
+	@Test
+	public void testDifferenceVertices() throws Exception{
+		/*Test  difference() method  by checking    the output  for getVertices()   on  the resultant   graph
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(TestGraphUtils.getLongLongVertexDataDifference(env),
+				TestGraphUtils.getLongLongEdgeDataDifference(env), env);
+
+		graph = graph.difference(graph2);
+
+		List<Vertex<Long, Long>> result = graph.getVertices().collect();
+
+		expectedResult =  "2,2\n" +
+				"4,4\n" +
+				"5,5\n" ;
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testDifference2() throws Exception {
+		/*
+		 * Test difference() such that no common vertices are there
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Vertex<Long, Long>> vertex = env.fromElements(new Vertex<Long, Long>(6L, 6L));
+
+		Graph<Long, Long, Long> graph2 = Graph.fromDataSet(vertex,TestGraphUtils.getLongLongEdgeDataDifference2(env),env);
+
+		graph = graph.difference(graph2);
+
+		List<Edge<Long, Long>> result = graph.getEdges().collect();
+
+		expectedResult =	"1,2,12\n" +
+				"1,3,13\n" +
+				"2,3,23\n" +
+				"3,4,34\n" +
+				"3,5,35\n" +
+				"4,5,45\n" +
+				"5,1,51\n" ;
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testTriplets() throws Exception {
+		/*
+		 * Test getTriplets()
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+        DataSet<Triplet<Long,Long,Long>> data = graph.getTriplets();
+        List<Triplet<Long,Long,Long>> result= data.collect();
+
+		expectedResult = "1,2,1,2,12\n" + "1,3,1,3,13\n" +
+				"2,3,2,3,23\n" + "3,4,3,4,34\n" +
+				"3,5,3,5,35\n" + "4,5,4,5,45\n" +
+				"5,1,5,1,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
new file mode 100644
index 0000000..e406ce2
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithEdgesITCase.java
@@ -0,0 +1,532 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.api.java.tuple.Tuple3;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
+import org.apache.flink.graph.utils.EdgeToTuple3Map;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class JoinWithEdgesITCase extends MultipleProgramsTestBase {
+
+	public JoinWithEdgesITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testWithEdgesInputDataset() throws Exception {
+		/*
+		 * Test joinWithEdges with the input DataSet parameter identical
+		 * to the edge DataSet
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges()
+                        .map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,68\n" +
+	                "3,5,70\n" +
+	                "4,5,90\n" +
+	                "5,1,102\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithLessElements() throws Exception {
+	    /*
+		 * Test joinWithEdges with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet, but of the same type
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges().first(3)
+                        .map(new EdgeToTuple3Map<Long, Long>()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithLessElementsDifferentType() throws Exception {
+	    /*
+		 * Test joinWithEdges with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet and of a different type(Boolean)
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdges(graph.getEdges().first(3)
+                        .map(new BooleanEdgeValueMapper()), new DoubleIfTrueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithNoCommonKeys() throws Exception {
+	    /*
+		 * Test joinWithEdges with the input DataSet containing different keys than the edge DataSet
+		 * - the iterator becomes empty.
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdges(TestGraphUtils.getLongLongLongTuple3Data(env),
+                new DoubleValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,68\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithCustomType() throws Exception {
+	    /*
+	     * Test joinWithEdges with a DataSet containing custom parametrised type input values
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdges(TestGraphUtils.getLongLongCustomTuple3Data(env),
+                new CustomValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,10\n" +
+	                "1,3,20\n" +
+	                "2,3,30\n" +
+	                "3,4,40\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithEdgesOnSource() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnSource with the input DataSet parameter identical
+		 * to the edge DataSet
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges()
+                        .map(new ProjectSourceAndValueMapper()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,25\n" +
+	                "2,3,46\n" +
+	                "3,4,68\n" +
+	                "3,5,69\n" +
+	                "4,5,90\n" +
+	                "5,1,102\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnSourceWithLessElements() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet, but of the same type
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges().first(3)
+                        .map(new ProjectSourceAndValueMapper()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,25\n" +
+	                "2,3,46\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnSourceWithDifferentType() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnSource with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet and of a different type(Boolean)
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(graph.getEdges().first(3)
+                        .map(new ProjectSourceWithTrueMapper()), new DoubleIfTrueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnSourceWithNoCommonKeys() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnSource with the input DataSet containing different keys than the edge DataSet
+		 * - the iterator becomes empty.
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(TestGraphUtils.getLongLongTuple2SourceData(env),
+                new DoubleValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,20\n" +
+	                "1,3,20\n" +
+	                "2,3,60\n" +
+	                "3,4,80\n" +
+	                "3,5,80\n" +
+	                "4,5,120\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnSourceWithCustom() throws Exception {
+	    /*
+	     * Test joinWithEdgesOnSource with a DataSet containing custom parametrised type input values
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnSource(TestGraphUtils.getLongCustomTuple2SourceData(env),
+                new CustomValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,10\n" +
+	                "1,3,10\n" +
+	                "2,3,30\n" +
+	                "3,4,40\n" +
+	                "3,5,40\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithEdgesOnTarget() throws Exception {
+    /*
+	 * Test joinWithEdgesOnTarget with the input DataSet parameter identical
+	 * to the edge DataSet
+	 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges()
+                        .map(new ProjectTargetAndValueMapper()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,36\n" +
+	                "3,4,68\n" +
+	                "3,5,70\n" +
+	                "4,5,80\n" +
+	                "5,1,102\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithOnTargetWithLessElements() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet, but of the same type
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges().first(3)
+                        .map(new ProjectTargetAndValueMapper()), new AddValuesMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,36\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnTargetWithDifferentType() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnTarget with the input DataSet passed as a parameter containing
+		 * less elements than the edge DataSet and of a different type(Boolean)
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(graph.getEdges().first(3)
+                        .map(new ProjectTargetWithTrueMapper()), new DoubleIfTrueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,24\n" +
+	                "1,3,26\n" +
+	                "2,3,46\n" +
+	                "3,4,34\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnTargetWithNoCommonKeys() throws Exception {
+	    /*
+		 * Test joinWithEdgesOnTarget with the input DataSet containing different keys than the edge DataSet
+		 * - the iterator becomes empty.
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongLongTuple2TargetData(env),
+                new DoubleValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,20\n" +
+	                "1,3,40\n" +
+	                "2,3,40\n" +
+	                "3,4,80\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,140\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testOnTargetWithCustom() throws Exception {
+	    /*
+	     * Test joinWithEdgesOnTarget with a DataSet containing custom parametrised type input values
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithEdgesOnTarget(TestGraphUtils.getLongCustomTuple2TargetData(env),
+                new CustomValueMapper());
+
+        DataSet<Edge<Long,Long>> data = res.getEdges();
+        List<Edge<Long, Long>> result= data.collect();
+
+        expectedResult = "1,2,10\n" +
+	                "1,3,20\n" +
+	                "2,3,20\n" +
+	                "3,4,40\n" +
+	                "3,5,35\n" +
+	                "4,5,45\n" +
+	                "5,1,51\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@SuppressWarnings("serial")
+	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
+		public Long map(Tuple2<Long, Long> tuple) throws Exception {
+			return tuple.f0 + tuple.f1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class BooleanEdgeValueMapper implements MapFunction<Edge<Long, Long>, Tuple3<Long, Long, Boolean>> {
+        public Tuple3<Long, Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
+            return new Tuple3<Long, Long, Boolean>(edge.getSource(),
+                    edge.getTarget(), true);
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
+        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
+            if(tuple.f1) {
+                return tuple.f0 * 2;
+            }
+            else {
+                return tuple.f0;
+            }
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class DoubleValueMapper implements MapFunction<Tuple2<Long, Long>, Long> {
+        public Long map(Tuple2<Long, Long> tuple) throws Exception {
+            return tuple.f1 * 2;
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
+        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
+            return (long) tuple.f1.getIntField();
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class ProjectSourceAndValueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Long>> {
+        public Tuple2<Long, Long> map(Edge<Long, Long> edge) throws Exception {
+            return new Tuple2<Long, Long>(edge.getSource(), edge.getValue());
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class ProjectSourceWithTrueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Boolean>> {
+        public Tuple2<Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
+            return new Tuple2<Long, Boolean>(edge.getSource(), true);
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class ProjectTargetAndValueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Long>> {
+        public Tuple2<Long, Long> map(Edge<Long, Long> edge) throws Exception {
+            return new Tuple2<Long, Long>(edge.getTarget(), edge.getValue());
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class ProjectTargetWithTrueMapper implements MapFunction<Edge<Long, Long>, Tuple2<Long, Boolean>> {
+        public Tuple2<Long, Boolean> map(Edge<Long, Long> edge) throws Exception {
+            return new Tuple2<Long, Boolean>(edge.getTarget(), true);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
new file mode 100644
index 0000000..22a5535
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/JoinWithVerticesITCase.java
@@ -0,0 +1,214 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
+import org.apache.flink.graph.utils.VertexToTuple2Map;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class JoinWithVerticesITCase extends MultipleProgramsTestBase {
+
+	public JoinWithVerticesITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testJoinWithVertexSet() throws Exception {
+		/*
+		 * Test joinWithVertices with the input DataSet parameter identical
+		 * to the vertex DataSet
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices()
+                        .map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+       expectedResult = "1,2\n" +
+	                "2,4\n" +
+	                "3,6\n" +
+	                "4,8\n" +
+	                "5,10\n";
+       
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithLessElements() throws Exception {
+	/*
+	 * Test joinWithVertices with the input DataSet passed as a parameter containing
+	 * less elements than the vertex DataSet, but of the same type
+	 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices().first(3)
+                        .map(new VertexToTuple2Map<Long, Long>()), new AddValuesMapper());
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+        expectedResult = "1,2\n" +
+	                "2,4\n" +
+	                "3,6\n" +
+	                "4,4\n" +
+	                "5,5\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithDifferentType() throws Exception {
+	/*
+	 * Test joinWithVertices with the input DataSet passed as a parameter containing
+	 * less elements than the vertex DataSet and of a different type(Boolean)
+	 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithVertices(graph.getVertices().first(3)
+                        .map(new ProjectIdWithTrue()), new DoubleIfTrueMapper());
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+        expectedResult = "1,2\n" +
+	                "2,4\n" +
+	                "3,6\n" +
+	                "4,4\n" +
+	                "5,5\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithDifferentKeys() throws Exception {
+		/*
+		 * Test joinWithVertices with an input DataSet containing different keys than the vertex DataSet
+		 * - the iterator becomes empty.
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithVertices(TestGraphUtils.getLongLongTuple2Data(env),
+                new ProjectSecondMapper());
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+        expectedResult = "1,10\n" +
+	                "2,20\n" +
+	                "3,30\n" +
+	                "4,40\n" +
+	                "5,5\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@Test
+	public void testWithCustomType() throws Exception {
+		/*
+		 * Test joinWithVertices with a DataSet containing custom parametrised type input values
+		 */
+        final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+        Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+                TestGraphUtils.getLongLongEdgeData(env), env);
+
+        Graph<Long, Long, Long> res = graph.joinWithVertices(TestGraphUtils.getLongCustomTuple2Data(env),
+                new CustomValueMapper());
+
+		DataSet<Vertex<Long,Long>> data = res.getVertices();
+        List<Vertex<Long,Long>> result= data.collect();
+
+        expectedResult = "1,10\n" +
+	                "2,20\n" +
+	                "3,30\n" +
+	                "4,40\n" +
+	                "5,5\n";
+        
+		compareResultAsTuples(result, expectedResult);
+    }
+
+	@SuppressWarnings("serial")
+	private static final class AddValuesMapper implements MapFunction<Tuple2<Long, Long>, Long> {
+		public Long map(Tuple2<Long, Long> tuple) throws Exception {
+			return tuple.f0 + tuple.f1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ProjectIdWithTrue implements MapFunction<Vertex<Long, Long>, Tuple2<Long, Boolean>> {
+        public Tuple2<Long, Boolean> map(Vertex<Long, Long> vertex) throws Exception {
+            return new Tuple2<Long, Boolean>(vertex.getId(), true);
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class DoubleIfTrueMapper implements MapFunction<Tuple2<Long, Boolean>, Long> {
+        public Long map(Tuple2<Long, Boolean> tuple) throws Exception {
+            if(tuple.f1) {
+                return tuple.f0 * 2;
+            }
+            else {
+                return tuple.f0;
+            }
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class ProjectSecondMapper implements MapFunction<Tuple2<Long, Long>, Long> {
+        public Long map(Tuple2<Long, Long> tuple) throws Exception {
+            return tuple.f1;
+        }
+    }
+
+	@SuppressWarnings("serial")
+	private static final class CustomValueMapper implements MapFunction<Tuple2<Long, DummyCustomParameterizedType<Float>>, Long> {
+        public Long map(Tuple2<Long, DummyCustomParameterizedType<Float>> tuple) throws Exception {
+            return (long) tuple.f1.getIntField();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
new file mode 100644
index 0000000..35f7b0e
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapEdgesITCase.java
@@ -0,0 +1,210 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.Tuple1;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class MapEdgesITCase extends MultipleProgramsTestBase {
+
+	public MapEdgesITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testWithSameValue() throws Exception {
+		/*
+		 * Test mapEdges() keeping the same value type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long, Long>> mappedEdges = graph.mapEdges(new AddOneMapper()).getEdges();
+        List<Edge<Long, Long>> result= mappedEdges.collect();
+        
+		expectedResult = "1,2,13\n" +
+				"1,3,14\n" +
+				"2,3,24\n" +
+				"3,4,35\n" +
+				"3,5,36\n" + 
+				"4,5,46\n" + 
+				"5,1,52\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithStringValue() throws Exception {
+		/*
+		 * Test mapEdges() and change the value type to String
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long, String>> mappedEdges = graph.mapEdges(new ToStringMapper()).getEdges();
+		List<Edge<Long, String>> result= mappedEdges.collect();
+		
+		expectedResult = "1,2,string(12)\n" +
+				"1,3,string(13)\n" +
+				"2,3,string(23)\n" +
+				"3,4,string(34)\n" +
+				"3,5,string(35)\n" + 
+				"4,5,string(45)\n" + 
+				"5,1,string(51)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithTuple1Type() throws Exception {
+		/*
+		 * Test mapEdges() and change the value type to a Tuple1
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long, Tuple1<Long>>> mappedEdges = graph.mapEdges(new ToTuple1Mapper()).getEdges();
+		List<Edge<Long, Tuple1<Long>>> result= mappedEdges.collect();
+
+		expectedResult = "1,2,(12)\n" +
+				"1,3,(13)\n" +
+				"2,3,(23)\n" +
+				"3,4,(34)\n" +
+				"3,5,(35)\n" + 
+				"4,5,(45)\n" + 
+				"5,1,(51)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithCustomType() throws Exception {
+		/*
+		 * Test mapEdges() and change the value type to a custom type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long, DummyCustomType>> mappedEdges = graph.mapEdges(new ToCustomTypeMapper()).getEdges();
+		List<Edge<Long, DummyCustomType>> result= mappedEdges.collect();
+
+		expectedResult = "1,2,(T,12)\n" +
+			"1,3,(T,13)\n" +
+			"2,3,(T,23)\n" +
+			"3,4,(T,34)\n" +
+			"3,5,(T,35)\n" + 
+			"4,5,(T,45)\n" + 
+			"5,1,(T,51)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithParametrizedCustomType() throws Exception {
+		/*
+		 * Test mapEdges() and change the value type to a parameterized custom type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Edge<Long, DummyCustomParameterizedType<Double>>> mappedEdges = graph.mapEdges(
+				new ToCustomParametrizedTypeMapper()).getEdges();
+		List<Edge<Long, DummyCustomParameterizedType<Double>>> result= mappedEdges.collect();
+	
+		expectedResult = "1,2,(12.0,12)\n" +
+			"1,3,(13.0,13)\n" +
+			"2,3,(23.0,23)\n" +
+			"3,4,(34.0,34)\n" +
+			"3,5,(35.0,35)\n" + 
+			"4,5,(45.0,45)\n" + 
+			"5,1,(51.0,51)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AddOneMapper implements MapFunction<Edge<Long, Long>, Long> {
+		public Long map(Edge<Long, Long> edge) throws Exception {
+			return edge.getValue()+1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToStringMapper implements MapFunction<Edge<Long, Long>, String> {
+		public String map(Edge<Long, Long> edge) throws Exception {
+			return String.format("string(%d)", edge.getValue());
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToTuple1Mapper implements MapFunction<Edge<Long, Long>, Tuple1<Long>> {
+		public Tuple1<Long> map(Edge<Long, Long> edge) throws Exception {
+			Tuple1<Long> tupleValue = new Tuple1<Long>();
+			tupleValue.setFields(edge.getValue());
+			return tupleValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToCustomTypeMapper implements MapFunction<Edge<Long, Long>, DummyCustomType> {
+		public DummyCustomType map(Edge<Long, Long> edge) throws Exception {
+			DummyCustomType dummyValue = new DummyCustomType();
+			dummyValue.setIntField(edge.getValue().intValue());						
+			return dummyValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToCustomParametrizedTypeMapper implements MapFunction<Edge<Long, Long>, 
+		DummyCustomParameterizedType<Double>> {
+
+		public DummyCustomParameterizedType<Double> map(Edge<Long, Long> edge) throws Exception {
+			DummyCustomParameterizedType<Double> dummyValue = new DummyCustomParameterizedType<Double>();
+			dummyValue.setIntField(edge.getValue().intValue());
+			dummyValue.setTField(new Double(edge.getValue()));						
+			return dummyValue;
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
new file mode 100644
index 0000000..677a03c
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/MapVerticesITCase.java
@@ -0,0 +1,219 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.Tuple1;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomParameterizedType;
+import org.apache.flink.graph.test.TestGraphUtils.DummyCustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class MapVerticesITCase extends MultipleProgramsTestBase {
+
+	public MapVerticesITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testWithSameValue() throws Exception {
+		/*
+		 * Test mapVertices() keeping the same value type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Vertex<Long, Long>> mappedVertices = graph.mapVertices(new AddOneMapper()).getVertices();	
+        List<Vertex<Long, Long>> result= mappedVertices.collect();
+        
+		expectedResult = "1,2\n" +
+			"2,3\n" +
+			"3,4\n" +
+			"4,5\n" +
+			"5,6\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithStringValue() throws Exception {
+		/*
+		 * Test mapVertices() and change the value type to String
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Vertex<Long, String>> mappedVertices = graph.mapVertices(new ToStringMapper()).getVertices();
+        List<Vertex<Long, String>> result= mappedVertices.collect();
+
+		expectedResult = "1,one\n" +
+			"2,two\n" +
+			"3,three\n" +
+			"4,four\n" +
+			"5,five\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithtuple1Value() throws Exception {
+		/*
+		 * Test mapVertices() and change the value type to a Tuple1
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Vertex<Long, Tuple1<Long>>> mappedVertices = graph.mapVertices(new ToTuple1Mapper()).getVertices();
+        List<Vertex<Long, Tuple1<Long>>> result= mappedVertices.collect();
+
+		expectedResult = "1,(1)\n" +
+			"2,(2)\n" +
+			"3,(3)\n" +
+			"4,(4)\n" +
+			"5,(5)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithCustomType() throws Exception {
+		/*
+		 * Test mapVertices() and change the value type to a custom type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Vertex<Long, DummyCustomType>> mappedVertices = graph.mapVertices(new ToCustomTypeMapper()).getVertices();
+        List<Vertex<Long, DummyCustomType>> result= mappedVertices.collect();
+
+		expectedResult = "1,(T,1)\n" +
+			"2,(T,2)\n" +
+			"3,(T,3)\n" +
+			"4,(T,4)\n" +
+			"5,(T,5)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testWithCustomParametrizedType() throws Exception {
+		/*
+		 * Test mapVertices() and change the value type to a parameterized custom type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+		
+		DataSet<Vertex<Long, DummyCustomParameterizedType<Double>>> mappedVertices = graph.mapVertices(
+				new ToCustomParametrizedTypeMapper()).getVertices();
+        List<Vertex<Long, DummyCustomParameterizedType<Double>>> result= mappedVertices.collect();
+	
+		expectedResult = "1,(1.0,1)\n" +
+			"2,(2.0,2)\n" +
+			"3,(3.0,3)\n" +
+			"4,(4.0,4)\n" +
+			"5,(5.0,5)\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class AddOneMapper implements MapFunction<Vertex<Long, Long>, Long> {
+		public Long map(Vertex<Long, Long> value) throws Exception {
+			return value.getValue()+1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToStringMapper implements MapFunction<Vertex<Long, Long>, String> {
+		public String map(Vertex<Long, Long> vertex) throws Exception {
+			String stringValue;
+			if (vertex.getValue() == 1) {
+				stringValue = "one";
+			}
+			else if (vertex.getValue() == 2) {
+				stringValue = "two";
+			}
+			else if (vertex.getValue() == 3) {
+				stringValue = "three";
+			}
+			else if (vertex.getValue() == 4) {
+				stringValue = "four";
+			}
+			else if (vertex.getValue() == 5) {
+				stringValue = "five";
+			}
+			else {
+				stringValue = "";
+			}
+			return stringValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToTuple1Mapper implements MapFunction<Vertex<Long, Long>, Tuple1<Long>> {
+		public Tuple1<Long> map(Vertex<Long, Long> vertex) throws Exception {
+			Tuple1<Long> tupleValue = new Tuple1<Long>();
+			tupleValue.setFields(vertex.getValue());
+			return tupleValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToCustomTypeMapper implements MapFunction<Vertex<Long, Long>, DummyCustomType> {
+		public DummyCustomType map(Vertex<Long, Long> vertex) throws Exception {
+			DummyCustomType dummyValue = new DummyCustomType();
+			dummyValue.setIntField(vertex.getValue().intValue());						
+			return dummyValue;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class ToCustomParametrizedTypeMapper implements MapFunction<Vertex<Long, Long>, 
+		DummyCustomParameterizedType<Double>> {
+		
+		public DummyCustomParameterizedType<Double> map(Vertex<Long, Long> vertex) throws Exception {
+			DummyCustomParameterizedType<Double> dummyValue = new DummyCustomParameterizedType<Double>();
+			dummyValue.setIntField(vertex.getValue().intValue());
+			dummyValue.setTField(new Double(vertex.getValue()));						
+			return dummyValue;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
new file mode 100644
index 0000000..3bb19fa
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesMethodsITCase.java
@@ -0,0 +1,615 @@
+/*
+ * 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.operations;
+
+import java.util.List;
+
+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.EdgeDirection;
+import org.apache.flink.graph.EdgesFunction;
+import org.apache.flink.graph.EdgesFunctionWithVertexValue;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.ReduceEdgesFunction;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class ReduceOnEdgesMethodsITCase extends MultipleProgramsTestBase {
+
+	public ReduceOnEdgesMethodsITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+    private String expectedResult;
+
+	@Test
+	public void testLowestWeightOutNeighbor() throws Exception {
+		/*
+		 * Get the lowest-weight out-neighbor
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
+				graph.groupReduceOnEdges(new SelectMinWeightNeighbor(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
+
+	
+		expectedResult = "1,2\n" +
+				"2,3\n" +
+				"3,4\n" +
+				"4,5\n" +
+				"5,1\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testLowestWeightInNeighbor() throws Exception {
+		/*
+		 * Get the lowest-weight in-neighbor
+		 * for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
+				graph.groupReduceOnEdges(new SelectMinWeightInNeighbor(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
+
+		expectedResult = "1,5\n" +
+					"2,1\n" + 
+					"3,1\n" +
+					"4,3\n" + 
+					"5,3\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllOutNeighbors() throws Exception {
+		/*
+		 * Get the all the out-neighbors for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
+				graph.groupReduceOnEdges(new SelectOutNeighbors(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
+
+		expectedResult = "1,2\n" +
+				"1,3\n" +
+				"2,3\n" +
+				"3,4\n" +
+				"3,5\n" +
+				"4,5\n" +
+				"5,1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllOutNeighborsNoValue() throws Exception {
+		/*
+		 * Get the all the out-neighbors for each vertex except for the vertex with id 5.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
+				graph.groupReduceOnEdges(new SelectOutNeighborsExcludeFive(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
+
+		expectedResult = "1,2\n" +
+				"1,3\n" +
+				"2,3\n" +
+				"3,4\n" +
+				"3,5\n" +
+				"4,5";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllOutNeighborsWithValueGreaterThanTwo() throws Exception {
+		/*
+		 * Get the all the out-neighbors for each vertex that have a value greater than two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllOutNeighbors =
+				graph.groupReduceOnEdges(new SelectOutNeighborsValueGreaterThanTwo(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithAllOutNeighbors.collect();
+
+		expectedResult = "3,4\n" +
+				"3,5\n" +
+				"4,5\n" +
+				"5,1";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllInNeighbors() throws Exception {
+		/*
+		 * Get the all the in-neighbors for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
+				graph.groupReduceOnEdges(new SelectInNeighbors(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
+
+		expectedResult = "1,5\n" +
+				"2,1\n" +
+				"3,1\n" +
+				"3,2\n" +
+				"4,3\n" +
+				"5,3\n" +
+				"5,4";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllInNeighborsNoValue() throws Exception {
+		/*
+		 * Get the all the in-neighbors for each vertex except for the vertex with id 5.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
+				graph.groupReduceOnEdges(new SelectInNeighborsExceptFive(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
+
+		expectedResult = "1,5\n" +
+				"2,1\n" +
+				"3,1\n" +
+				"3,2\n" +
+				"4,3";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllInNeighborsWithValueGreaterThanTwo() throws Exception {
+		/*
+		 * Get the all the in-neighbors for each vertex that have a value greater than two.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllInNeighbors =
+				graph.groupReduceOnEdges(new SelectInNeighborsValueGreaterThanTwo(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithAllInNeighbors.collect();
+
+		expectedResult = "3,1\n" +
+				"3,2\n" +
+				"4,3\n" +
+				"5,3\n" +
+				"5,4";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllNeighbors() throws Exception {
+		/*
+		 * Get the all the neighbors for each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
+				graph.groupReduceOnEdges(new SelectNeighbors(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
+
+		expectedResult = "1,2\n" +
+				"1,3\n" +
+				"1,5\n" +
+				"2,1\n" +
+				"2,3\n" +
+				"3,1\n" +
+				"3,2\n" +
+				"3,4\n" +
+				"3,5\n" +
+				"4,3\n" +
+				"4,5\n" +
+				"5,1\n" +
+				"5,3\n" +
+				"5,4";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllNeighborsNoValue() throws Exception {
+		/*
+		 * Get the all the neighbors for each vertex except for vertices with id 5 and 2.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
+				graph.groupReduceOnEdges(new SelectNeighborsExceptFiveAndTwo(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
+
+		expectedResult = "1,2\n" +
+				"1,3\n" +
+				"1,5\n" +
+				"3,1\n" +
+				"3,2\n" +
+				"3,4\n" +
+				"3,5\n" +
+				"4,3\n" +
+				"4,5";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testAllNeighborsWithValueGreaterThanFour() throws Exception {
+		/*
+		 * Get the all the neighbors for each vertex that have a value greater than four.
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
+				graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithAllNeighbors.collect();
+
+		expectedResult = "5,1\n" +
+				"5,3\n" +
+				"5,4";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testMaxWeightEdge() throws Exception {
+		/*
+		 * Get the maximum weight among all edges
+		 * of a vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithMaxEdgeWeight = 
+				graph.groupReduceOnEdges(new SelectMaxWeightNeighbor(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithMaxEdgeWeight.collect();
+
+		expectedResult = "1,51\n" +
+				"2,23\n" + 
+				"3,35\n" +
+				"4,45\n" + 
+				"5,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testLowestWeightOutNeighborNoValue() throws Exception {
+		/*
+		 * Get the lowest-weight out of all the out-neighbors
+		 * of each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
+				graph.reduceOnEdges(new SelectMinWeightNeighborNoValue(), EdgeDirection.OUT);
+		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
+
+		expectedResult = "1,12\n" +
+				"2,23\n" +
+				"3,34\n" +
+				"4,45\n" +
+				"5,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testLowestWeightInNeighborNoValue() throws Exception {
+		/*
+		 * Get the lowest-weight out of all the in-neighbors
+		 * of each vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithLowestOutNeighbor = 
+				graph.reduceOnEdges(new SelectMinWeightNeighborNoValue(), EdgeDirection.IN);
+		List<Tuple2<Long,Long>> result = verticesWithLowestOutNeighbor.collect();
+
+		expectedResult = "1,51\n" +
+				"2,12\n" +
+				"3,13\n" +
+				"4,34\n" +
+				"5,35\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@Test
+	public void testMaxWeightAllNeighbors() throws Exception {
+		/*
+		 * Get the maximum weight among all edges
+		 * of a vertex
+         */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env), 
+				TestGraphUtils.getLongLongEdgeData(env), env);
+
+		DataSet<Tuple2<Long, Long>> verticesWithMaxEdgeWeight = 
+				graph.reduceOnEdges(new SelectMaxWeightNeighborNoValue(), EdgeDirection.ALL);
+		List<Tuple2<Long,Long>> result = verticesWithMaxEdgeWeight.collect();
+
+		expectedResult = "1,51\n" +
+				"2,23\n" + 
+				"3,35\n" +
+				"4,45\n" + 
+				"5,51\n";
+		
+		compareResultAsTuples(result, expectedResult);
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinWeightNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v,
+				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
+			
+			long weight = Long.MAX_VALUE;
+			long minNeighborId = 0;
+
+			for (Edge<Long, Long> edge: edges) {
+				if (edge.getValue() < weight) {
+					weight = edge.getValue();
+					minNeighborId = edge.getTarget();
+				}
+			}
+			out.collect(new Tuple2<Long, Long>(v.getId(), minNeighborId));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectMaxWeightNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v,
+				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
+			
+			long weight = Long.MIN_VALUE;
+
+			for (Edge<Long, Long> edge: edges) {
+				if (edge.getValue() > weight) {
+					weight = edge.getValue();
+				}
+			}
+			out.collect(new Tuple2<Long, Long>(v.getId(), weight));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinWeightNeighborNoValue implements ReduceEdgesFunction<Long> {
+
+		@Override
+		public Long reduceEdges(Long firstEdgeValue, Long secondEdgeValue) {
+			return Math.min(firstEdgeValue, secondEdgeValue);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectMaxWeightNeighborNoValue implements ReduceEdgesFunction<Long> {
+
+		@Override
+		public Long reduceEdges(Long firstEdgeValue, Long secondEdgeValue) {
+			return Math.max(firstEdgeValue, secondEdgeValue);
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectMinWeightInNeighbor implements EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v,
+				Iterable<Edge<Long, Long>> edges, Collector<Tuple2<Long, Long>> out) throws Exception {
+			
+			long weight = Long.MAX_VALUE;
+			long minNeighborId = 0;
+			
+			for (Edge<Long, Long> edge: edges) {
+				if (edge.getValue() < weight) {
+					weight = edge.getValue();
+					minNeighborId = edge.getSource();
+				}
+			}
+			out.collect(new Tuple2<Long, Long>(v.getId(), minNeighborId));
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectOutNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectOutNeighborsExcludeFive implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				if(edge.f0 != 5) {
+					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectOutNeighborsValueGreaterThanTwo implements
+			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for (Edge<Long, Long> edge: edges) {
+				if(v.getValue() > 2) {
+					out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectInNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectInNeighborsExceptFive implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+
+			for(Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				if(edge.f0 != 5) {
+					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectInNeighborsValueGreaterThanTwo implements
+			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for (Edge<Long, Long> edge: edges) {
+				if(v.getValue() > 2) {
+					out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectNeighbors implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for (Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				if (edge.f0 == edge.f1.getTarget()) {
+					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
+				} else {
+					out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectNeighborsExceptFiveAndTwo implements EdgesFunction<Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Iterable<Tuple2<Long, Edge<Long, Long>>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for (Tuple2<Long, Edge<Long, Long>> edge : edges) {
+				if(edge.f0 != 5 && edge.f0 != 2) {
+					if (edge.f0 == edge.f1.getTarget()) {
+						out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getSource()));
+					} else {
+						out.collect(new Tuple2<Long, Long>(edge.f0, edge.f1.getTarget()));
+					}
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static final class SelectNeighborsValueGreaterThanFour implements
+			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for(Edge<Long, Long> edge : edges) {
+				if(v.getValue() > 4) {
+					if(v.getId().equals(edge.getTarget())) {
+						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
+					} else {
+						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
+					}
+				}
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
new file mode 100644
index 0000000..ab10947
--- /dev/null
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -0,0 +1,142 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.graph.Edge;
+import org.apache.flink.graph.EdgeDirection;
+import org.apache.flink.graph.EdgesFunctionWithVertexValue;
+import org.apache.flink.graph.Graph;
+import org.apache.flink.graph.Vertex;
+import org.apache.flink.graph.test.TestGraphUtils;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.util.Collector;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+public class ReduceOnEdgesWithExceptionITCase {
+
+	private static final int PARALLELISM = 4;
+
+	private static ForkableFlinkMiniCluster cluster;
+
+
+	@BeforeClass
+	public static void setupCluster() {
+		try {
+			Configuration config = new Configuration();
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster.start();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail("Error starting test cluster: " + e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void tearDownCluster() {
+		try {
+			cluster.stop();
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Cluster shutdown caused an exception: " + t.getMessage());
+		}
+	}
+
+	/**
+	 * Test groupReduceOnEdges() with an edge having a srcId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnEdgesInvalidEdgeSrcId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidSrcData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
+					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
+
+			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+	/**
+	 * Test groupReduceOnEdges() with an edge having a trgId that does not exist in the vertex DataSet
+	 */
+	@Test
+	public void testGroupReduceOnEdgesInvalidEdgeTrgId() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				"localhost", cluster.getLeaderRPCPort());
+		env.setParallelism(PARALLELISM);
+		env.getConfig().disableSysoutLogging();
+
+		Graph<Long, Long, Long> graph = Graph.fromDataSet(TestGraphUtils.getLongLongVertexData(env),
+				TestGraphUtils.getLongLongEdgeInvalidTrgData(env), env);
+
+		try {
+			DataSet<Tuple2<Long, Long>> verticesWithAllNeighbors =
+					graph.groupReduceOnEdges(new SelectNeighborsValueGreaterThanFour(), EdgeDirection.ALL);
+
+			verticesWithAllNeighbors.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
+			env.execute();
+		} catch (Exception e) {
+			// We expect the job to fail with an exception
+		}
+	}
+
+
+	@SuppressWarnings("serial")
+	private static final class SelectNeighborsValueGreaterThanFour implements
+			EdgesFunctionWithVertexValue<Long, Long, Long, Tuple2<Long, Long>> {
+
+		@Override
+		public void iterateEdges(Vertex<Long, Long> v, Iterable<Edge<Long, Long>> edges,
+								 Collector<Tuple2<Long, Long>> out) throws Exception {
+			for(Edge<Long, Long> edge : edges) {
+				if(v.getValue() > 4) {
+					if(v.getId().equals(edge.getTarget())) {
+						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getSource()));
+					} else {
+						out.collect(new Tuple2<Long, Long>(v.getId(), edge.getTarget()));
+					}
+				}
+			}
+		}
+	}
+}


[24/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
[FLINK-2833] [gelly] create a flink-libraries module and move gelly there

This closes #1241


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

Branch: refs/heads/master
Commit: 91ffbc1e3b626b93398271a6ea10c57b18459339
Parents: 71d5a39
Author: vasia <va...@apache.org>
Authored: Thu Oct 8 11:48:06 2015 +0200
Committer: vasia <va...@apache.org>
Committed: Fri Oct 9 17:33:15 2015 +0200

----------------------------------------------------------------------
 docs/libs/gelly_guide.md                        |    6 +-
 flink-libraries/flink-gelly-scala/pom.xml       |  204 ++
 .../flink/graph/scala/EdgesFunction.scala       |   35 +
 .../scala/EdgesFunctionWithVertexValue.scala    |   33 +
 .../org/apache/flink/graph/scala/Graph.scala    | 1014 +++++++++
 .../flink/graph/scala/NeighborsFunction.scala   |   37 +
 .../NeighborsFunctionWithVertexValue.scala      |   40 +
 .../scala/example/ConnectedComponents.scala     |  121 ++
 .../example/GSASingleSourceShortestPaths.scala  |  156 ++
 .../graph/scala/example/GraphMetrics.scala      |  128 ++
 .../example/SingleSourceShortestPaths.scala     |  170 ++
 .../org/apache/flink/graph/scala/package.scala  |   30 +
 .../graph/scala/utils/EdgeToTuple3Map.scala     |   30 +
 .../graph/scala/utils/Tuple2ToVertexMap.scala   |   30 +
 .../graph/scala/utils/Tuple3ToEdgeMap.scala     |   30 +
 .../graph/scala/utils/VertexToTuple2Map.scala   |   30 +
 .../test/GellyScalaAPICompletenessTest.scala    |   43 +
 .../flink/graph/scala/test/TestGraphUtils.scala |   55 +
 .../scala/test/operations/DegreesITCase.scala   |   69 +
 .../operations/GraphCreationWithCsvITCase.scala |  225 ++
 .../test/operations/GraphMutationsITCase.scala  |  260 +++
 .../test/operations/GraphOperationsITCase.scala |  282 +++
 .../test/operations/JoinWithEdgesITCase.scala   |  151 ++
 .../operations/JoinWithVerticesITCase.scala     |   74 +
 .../scala/test/operations/MapEdgesITCase.scala  |   81 +
 .../test/operations/MapVerticesITCase.scala     |   76 +
 .../operations/ReduceOnEdgesMethodsITCase.scala |  150 ++
 .../ReduceOnNeighborMethodsITCase.scala         |  126 ++
 flink-libraries/flink-gelly/pom.xml             |   67 +
 .../main/java/org/apache/flink/graph/Edge.java  |   75 +
 .../org/apache/flink/graph/EdgeDirection.java   |   35 +
 .../org/apache/flink/graph/EdgesFunction.java   |   38 +
 .../graph/EdgesFunctionWithVertexValue.java     |   39 +
 .../main/java/org/apache/flink/graph/Graph.java | 1948 ++++++++++++++++++
 .../org/apache/flink/graph/GraphAlgorithm.java  |   30 +
 .../org/apache/flink/graph/GraphCsvReader.java  |  486 +++++
 .../flink/graph/IterationConfiguration.java     |  155 ++
 .../apache/flink/graph/NeighborsFunction.java   |   40 +
 .../graph/NeighborsFunctionWithVertexValue.java |   40 +
 .../apache/flink/graph/ReduceEdgesFunction.java |   34 +
 .../flink/graph/ReduceNeighborsFunction.java    |   35 +
 .../java/org/apache/flink/graph/Triplet.java    |   77 +
 .../java/org/apache/flink/graph/Vertex.java     |   56 +
 .../graph/example/ConnectedComponents.java      |  141 ++
 .../graph/example/EuclideanGraphWeighing.java   |  215 ++
 .../example/GSASingleSourceShortestPaths.java   |  191 ++
 .../flink/graph/example/GraphMetrics.java       |  170 ++
 .../flink/graph/example/IncrementalSSSP.java    |  268 +++
 .../graph/example/JaccardSimilarityMeasure.java |  214 ++
 .../flink/graph/example/MusicProfiles.java      |  302 +++
 .../example/SingleSourceShortestPaths.java      |  199 ++
 .../example/utils/CommunityDetectionData.java   |   95 +
 .../utils/ConnectedComponentsDefaultData.java   |   57 +
 .../graph/example/utils/EuclideanGraphData.java |   86 +
 .../flink/graph/example/utils/ExampleUtils.java |  162 ++
 .../example/utils/IncrementalSSSPData.java      |   95 +
 .../utils/JaccardSimilarityMeasureData.java     |   58 +
 .../example/utils/LabelPropagationData.java     |  114 +
 .../graph/example/utils/MusicProfilesData.java  |  108 +
 .../flink/graph/example/utils/PageRankData.java |   69 +
 .../utils/SingleSourceShortestPathsData.java    |   62 +
 .../graph/example/utils/TriangleCountData.java  |   56 +
 .../apache/flink/graph/gsa/ApplyFunction.java   |  155 ++
 .../flink/graph/gsa/GSAConfiguration.java       |  133 ++
 .../apache/flink/graph/gsa/GatherFunction.java  |  134 ++
 .../graph/gsa/GatherSumApplyIteration.java      |  425 ++++
 .../org/apache/flink/graph/gsa/Neighbor.java    |   45 +
 .../org/apache/flink/graph/gsa/SumFunction.java |  134 ++
 .../flink/graph/library/CommunityDetection.java |  183 ++
 .../graph/library/ConnectedComponents.java      |   97 +
 .../graph/library/GSAConnectedComponents.java   |   88 +
 .../apache/flink/graph/library/GSAPageRank.java |  135 ++
 .../library/GSASingleSourceShortestPaths.java   |  101 +
 .../flink/graph/library/GSATriangleCount.java   |  190 ++
 .../flink/graph/library/LabelPropagation.java   |  116 ++
 .../apache/flink/graph/library/PageRank.java    |  146 ++
 .../library/SingleSourceShortestPaths.java      |  112 +
 .../flink/graph/spargel/MessageIterator.java    |   58 +
 .../flink/graph/spargel/MessagingFunction.java  |  303 +++
 .../spargel/VertexCentricConfiguration.java     |  135 ++
 .../graph/spargel/VertexCentricIteration.java   |  686 ++++++
 .../graph/spargel/VertexUpdateFunction.java     |  253 +++
 .../flink/graph/utils/EdgeToTuple3Map.java      |   35 +
 .../flink/graph/utils/NullValueEdgeMapper.java  |   32 +
 .../flink/graph/utils/Tuple2ToVertexMap.java    |   35 +
 .../flink/graph/utils/Tuple3ToEdgeMap.java      |   41 +
 .../flink/graph/utils/VertexToTuple2Map.java    |   35 +
 .../flink/graph/validation/GraphValidator.java  |   37 +
 .../validation/InvalidVertexIdsValidator.java   |   74 +
 .../apache/flink/graph/gsa/GSACompilerTest.java |  146 ++
 .../flink/graph/gsa/GSATranslationTest.java     |  164 ++
 .../graph/spargel/SpargelCompilerTest.java      |  214 ++
 .../graph/spargel/SpargelTranslationTest.java   |  231 +++
 .../test/CollectionModeSuperstepITCase.java     |   84 +
 .../test/GatherSumApplyConfigurationITCase.java |  404 ++++
 .../flink/graph/test/GatherSumApplyITCase.java  |  106 +
 .../apache/flink/graph/test/TestGraphUtils.java |  417 ++++
 .../test/VertexCentricConfigurationITCase.java  |  689 +++++++
 .../test/example/ConnectedComponentsITCase.java |   71 +
 .../example/EuclideanGraphWeighingITCase.java   |   77 +
 .../test/example/IncrementalSSSPITCase.java     |  133 ++
 .../example/JaccardSimilarityMeasureITCase.java |   72 +
 .../graph/test/example/MusicProfilesITCase.java |  101 +
 .../SingleSourceShortestPathsITCase.java        |   81 +
 .../test/library/CommunityDetectionITCase.java  |   82 +
 ...ctedComponentsWithRandomisedEdgesITCase.java |   94 +
 .../test/library/LabelPropagationITCase.java    |   78 +
 .../graph/test/library/PageRankITCase.java      |  132 ++
 .../graph/test/library/TriangleCountITCase.java |   56 +
 .../graph/test/operations/DegreesITCase.java    |  178 ++
 .../operations/DegreesWithExceptionITCase.java  |  189 ++
 .../test/operations/FromCollectionITCase.java   |  118 ++
 .../test/operations/GraphCreationITCase.java    |  174 ++
 .../operations/GraphCreationWithCsvITCase.java  |  204 ++
 .../GraphCreationWithMapperITCase.java          |  158 ++
 .../test/operations/GraphMutationsITCase.java   |  603 ++++++
 .../test/operations/GraphOperationsITCase.java  |  378 ++++
 .../test/operations/JoinWithEdgesITCase.java    |  532 +++++
 .../test/operations/JoinWithVerticesITCase.java |  214 ++
 .../graph/test/operations/MapEdgesITCase.java   |  210 ++
 .../test/operations/MapVerticesITCase.java      |  219 ++
 .../operations/ReduceOnEdgesMethodsITCase.java  |  615 ++++++
 .../ReduceOnEdgesWithExceptionITCase.java       |  142 ++
 .../ReduceOnNeighborMethodsITCase.java          |  668 ++++++
 .../ReduceOnNeighborsWithExceptionITCase.java   |  203 ++
 flink-libraries/pom.xml                         |   40 +
 flink-staging/flink-gelly-scala/pom.xml         |  204 --
 .../flink/graph/scala/EdgesFunction.scala       |   35 -
 .../scala/EdgesFunctionWithVertexValue.scala    |   33 -
 .../org/apache/flink/graph/scala/Graph.scala    | 1014 ---------
 .../flink/graph/scala/NeighborsFunction.scala   |   37 -
 .../NeighborsFunctionWithVertexValue.scala      |   40 -
 .../scala/example/ConnectedComponents.scala     |  121 --
 .../example/GSASingleSourceShortestPaths.scala  |  156 --
 .../graph/scala/example/GraphMetrics.scala      |  128 --
 .../example/SingleSourceShortestPaths.scala     |  170 --
 .../org/apache/flink/graph/scala/package.scala  |   30 -
 .../graph/scala/utils/EdgeToTuple3Map.scala     |   30 -
 .../graph/scala/utils/Tuple2ToVertexMap.scala   |   30 -
 .../graph/scala/utils/Tuple3ToEdgeMap.scala     |   30 -
 .../graph/scala/utils/VertexToTuple2Map.scala   |   30 -
 .../test/GellyScalaAPICompletenessTest.scala    |   43 -
 .../flink/graph/scala/test/TestGraphUtils.scala |   55 -
 .../scala/test/operations/DegreesITCase.scala   |   69 -
 .../operations/GraphCreationWithCsvITCase.scala |  225 --
 .../test/operations/GraphMutationsITCase.scala  |  260 ---
 .../test/operations/GraphOperationsITCase.scala |  282 ---
 .../test/operations/JoinWithEdgesITCase.scala   |  151 --
 .../operations/JoinWithVerticesITCase.scala     |   74 -
 .../scala/test/operations/MapEdgesITCase.scala  |   81 -
 .../test/operations/MapVerticesITCase.scala     |   76 -
 .../operations/ReduceOnEdgesMethodsITCase.scala |  150 --
 .../ReduceOnNeighborMethodsITCase.scala         |  126 --
 flink-staging/flink-gelly/pom.xml               |   67 -
 .../main/java/org/apache/flink/graph/Edge.java  |   75 -
 .../org/apache/flink/graph/EdgeDirection.java   |   35 -
 .../org/apache/flink/graph/EdgesFunction.java   |   38 -
 .../graph/EdgesFunctionWithVertexValue.java     |   39 -
 .../main/java/org/apache/flink/graph/Graph.java | 1948 ------------------
 .../org/apache/flink/graph/GraphAlgorithm.java  |   30 -
 .../org/apache/flink/graph/GraphCsvReader.java  |  486 -----
 .../flink/graph/IterationConfiguration.java     |  155 --
 .../apache/flink/graph/NeighborsFunction.java   |   40 -
 .../graph/NeighborsFunctionWithVertexValue.java |   40 -
 .../apache/flink/graph/ReduceEdgesFunction.java |   34 -
 .../flink/graph/ReduceNeighborsFunction.java    |   35 -
 .../java/org/apache/flink/graph/Triplet.java    |   77 -
 .../java/org/apache/flink/graph/Vertex.java     |   56 -
 .../graph/example/ConnectedComponents.java      |  141 --
 .../graph/example/EuclideanGraphWeighing.java   |  215 --
 .../example/GSASingleSourceShortestPaths.java   |  191 --
 .../flink/graph/example/GraphMetrics.java       |  170 --
 .../flink/graph/example/IncrementalSSSP.java    |  268 ---
 .../graph/example/JaccardSimilarityMeasure.java |  214 --
 .../flink/graph/example/MusicProfiles.java      |  302 ---
 .../example/SingleSourceShortestPaths.java      |  199 --
 .../example/utils/CommunityDetectionData.java   |   95 -
 .../utils/ConnectedComponentsDefaultData.java   |   57 -
 .../graph/example/utils/EuclideanGraphData.java |   86 -
 .../flink/graph/example/utils/ExampleUtils.java |  162 --
 .../example/utils/IncrementalSSSPData.java      |   95 -
 .../utils/JaccardSimilarityMeasureData.java     |   58 -
 .../example/utils/LabelPropagationData.java     |  114 -
 .../graph/example/utils/MusicProfilesData.java  |  108 -
 .../flink/graph/example/utils/PageRankData.java |   69 -
 .../utils/SingleSourceShortestPathsData.java    |   62 -
 .../graph/example/utils/TriangleCountData.java  |   56 -
 .../apache/flink/graph/gsa/ApplyFunction.java   |  155 --
 .../flink/graph/gsa/GSAConfiguration.java       |  133 --
 .../apache/flink/graph/gsa/GatherFunction.java  |  134 --
 .../graph/gsa/GatherSumApplyIteration.java      |  425 ----
 .../org/apache/flink/graph/gsa/Neighbor.java    |   45 -
 .../org/apache/flink/graph/gsa/SumFunction.java |  134 --
 .../flink/graph/library/CommunityDetection.java |  183 --
 .../graph/library/ConnectedComponents.java      |   97 -
 .../graph/library/GSAConnectedComponents.java   |   88 -
 .../apache/flink/graph/library/GSAPageRank.java |  135 --
 .../library/GSASingleSourceShortestPaths.java   |  101 -
 .../flink/graph/library/GSATriangleCount.java   |  190 --
 .../flink/graph/library/LabelPropagation.java   |  116 --
 .../apache/flink/graph/library/PageRank.java    |  146 --
 .../library/SingleSourceShortestPaths.java      |  112 -
 .../flink/graph/spargel/MessageIterator.java    |   58 -
 .../flink/graph/spargel/MessagingFunction.java  |  303 ---
 .../spargel/VertexCentricConfiguration.java     |  135 --
 .../graph/spargel/VertexCentricIteration.java   |  686 ------
 .../graph/spargel/VertexUpdateFunction.java     |  253 ---
 .../flink/graph/utils/EdgeToTuple3Map.java      |   35 -
 .../flink/graph/utils/NullValueEdgeMapper.java  |   32 -
 .../flink/graph/utils/Tuple2ToVertexMap.java    |   35 -
 .../flink/graph/utils/Tuple3ToEdgeMap.java      |   41 -
 .../flink/graph/utils/VertexToTuple2Map.java    |   35 -
 .../flink/graph/validation/GraphValidator.java  |   37 -
 .../validation/InvalidVertexIdsValidator.java   |   74 -
 .../apache/flink/graph/gsa/GSACompilerTest.java |  146 --
 .../flink/graph/gsa/GSATranslationTest.java     |  164 --
 .../graph/spargel/SpargelCompilerTest.java      |  214 --
 .../graph/spargel/SpargelTranslationTest.java   |  231 ---
 .../test/CollectionModeSuperstepITCase.java     |   84 -
 .../test/GatherSumApplyConfigurationITCase.java |  404 ----
 .../flink/graph/test/GatherSumApplyITCase.java  |  106 -
 .../apache/flink/graph/test/TestGraphUtils.java |  417 ----
 .../test/VertexCentricConfigurationITCase.java  |  689 -------
 .../test/example/ConnectedComponentsITCase.java |   71 -
 .../example/EuclideanGraphWeighingITCase.java   |   77 -
 .../test/example/IncrementalSSSPITCase.java     |  133 --
 .../example/JaccardSimilarityMeasureITCase.java |   72 -
 .../graph/test/example/MusicProfilesITCase.java |  101 -
 .../SingleSourceShortestPathsITCase.java        |   81 -
 .../test/library/CommunityDetectionITCase.java  |   82 -
 ...ctedComponentsWithRandomisedEdgesITCase.java |   94 -
 .../test/library/LabelPropagationITCase.java    |   78 -
 .../graph/test/library/PageRankITCase.java      |  132 --
 .../graph/test/library/TriangleCountITCase.java |   56 -
 .../graph/test/operations/DegreesITCase.java    |  178 --
 .../operations/DegreesWithExceptionITCase.java  |  189 --
 .../test/operations/FromCollectionITCase.java   |  118 --
 .../test/operations/GraphCreationITCase.java    |  174 --
 .../operations/GraphCreationWithCsvITCase.java  |  204 --
 .../GraphCreationWithMapperITCase.java          |  158 --
 .../test/operations/GraphMutationsITCase.java   |  603 ------
 .../test/operations/GraphOperationsITCase.java  |  378 ----
 .../test/operations/JoinWithEdgesITCase.java    |  532 -----
 .../test/operations/JoinWithVerticesITCase.java |  214 --
 .../graph/test/operations/MapEdgesITCase.java   |  210 --
 .../test/operations/MapVerticesITCase.java      |  219 --
 .../operations/ReduceOnEdgesMethodsITCase.java  |  615 ------
 .../ReduceOnEdgesWithExceptionITCase.java       |  142 --
 .../ReduceOnNeighborMethodsITCase.java          |  668 ------
 .../ReduceOnNeighborsWithExceptionITCase.java   |  203 --
 flink-staging/pom.xml                           |    2 -
 pom.xml                                         |    1 +
 252 files changed, 21696 insertions(+), 21657 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/docs/libs/gelly_guide.md
----------------------------------------------------------------------
diff --git a/docs/libs/gelly_guide.md b/docs/libs/gelly_guide.md
index 766b395..f8e4f28 100644
--- a/docs/libs/gelly_guide.md
+++ b/docs/libs/gelly_guide.md
@@ -30,7 +30,7 @@ Gelly is a Graph API for Flink. It contains a set of methods and utilities which
 Using Gelly
 -----------
 
-Gelly is currently part of the *staging* Maven project. All relevant classes are located in the *org.apache.flink.graph* package.
+Gelly is currently part of the *libraries* Maven project. All relevant classes are located in the *org.apache.flink.graph* package.
 
 Add the following dependency to your `pom.xml` to use Gelly.
 
@@ -57,7 +57,7 @@ Add the following dependency to your `pom.xml` to use Gelly.
 
 Note that Gelly is currently not part of the binary distribution. See linking with it for cluster execution [here](../apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
 
-The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink DataSet API. After reading this guide, you might also want to check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ "Gelly examples" %}.
+The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink DataSet API. After reading this guide, you might also want to check the {% gh_link /flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/ "Gelly examples" %}.
 
 Graph Representation
 -----------
@@ -1194,7 +1194,7 @@ final class UpdateDistance extends ApplyFunction[Long, Double, Double] {
 
 Note that `gather` takes a `Neighbor` type as an argument. This is a convenience type which simply wraps a vertex with its neighboring edge.
 
-For more examples of how to implement algorithms with the Gather-Sum-Apply model, check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java "GSAPageRank" %} and {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java "GSAConnectedComponents" %} library methods of Gelly.
+For more examples of how to implement algorithms with the Gather-Sum-Apply model, check the {% gh_link /flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java "GSAPageRank" %} and {% gh_link /flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java "GSAConnectedComponents" %} library methods of Gelly.
 
 [Back to top](#top)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/pom.xml b/flink-libraries/flink-gelly-scala/pom.xml
new file mode 100644
index 0000000..90d2971
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/pom.xml
@@ -0,0 +1,204 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-libraries</artifactId>
+        <version>0.10-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>flink-gelly-scala</artifactId>
+
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-scala</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-clients</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-gelly</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <!-- Scala Compiler -->
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+                <version>3.1.4</version>
+                <executions>
+                    <!-- Run scala compiler in the process-resources phase, so that dependencies on
+                        scala classes can be resolved later in the (Java) compile phase -->
+                    <execution>
+                        <id>scala-compile-first</id>
+                        <phase>process-resources</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+
+                    <!-- Run scala compiler in the process-test-resources phase, so that dependencies on
+                         scala classes can be resolved later in the (Java) test-compile phase -->
+                    <execution>
+                        <id>scala-test-compile</id>
+                        <phase>process-test-resources</phase>
+                        <goals>
+                            <goal>testCompile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <jvmArgs>
+                        <jvmArg>-Xms128m</jvmArg>
+                        <jvmArg>-Xmx512m</jvmArg>
+                    </jvmArgs>
+                    <compilerPlugins combine.children="append">
+                        <compilerPlugin>
+                            <groupId>org.scalamacros</groupId>
+                            <artifactId>paradise_${scala.version}</artifactId>
+                            <version>${scala.macros.version}</version>
+                        </compilerPlugin>
+                    </compilerPlugins>
+                </configuration>
+            </plugin>
+
+            <!-- Eclipse Integration -->
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-eclipse-plugin</artifactId>
+                <version>2.8</version>
+                <configuration>
+                    <downloadSources>true</downloadSources>
+                    <projectnatures>
+                        <projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
+                        <projectnature>org.eclipse.jdt.core.javanature</projectnature>
+                    </projectnatures>
+                    <buildcommands>
+                        <buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
+                    </buildcommands>
+                    <classpathContainers>
+                        <classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
+                        <classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
+                    </classpathContainers>
+                    <excludes>
+                        <exclude>org.scala-lang:scala-library</exclude>
+                        <exclude>org.scala-lang:scala-compiler</exclude>
+                    </excludes>
+                    <sourceIncludes>
+                        <sourceInclude>**/*.scala</sourceInclude>
+                        <sourceInclude>**/*.java</sourceInclude>
+                    </sourceIncludes>
+                </configuration>
+            </plugin>
+
+            <!-- Adding scala source directories to build path -->
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.7</version>
+                <executions>
+                    <!-- Add src/main/scala to eclipse build path -->
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/scala</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                    <!-- Add src/test/scala to eclipse build path -->
+                    <execution>
+                        <id>add-test-source</id>
+                        <phase>generate-test-sources</phase>
+                        <goals>
+                            <goal>add-test-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/test/scala</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.scalastyle</groupId>
+                <artifactId>scalastyle-maven-plugin</artifactId>
+                <version>0.5.0</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <verbose>false</verbose>
+                    <failOnViolation>true</failOnViolation>
+                    <includeTestSourceDirectory>true</includeTestSourceDirectory>
+                    <failOnWarning>false</failOnWarning>
+                    <sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
+                    <testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
+                    <configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
+                    <outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
+                    <outputEncoding>UTF-8</outputEncoding>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
new file mode 100644
index 0000000..70a5fdf
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunction.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.java.tuple.Tuple2
+import org.apache.flink.graph.Edge
+import org.apache.flink.util.Collector
+
+abstract class EdgesFunction[K, EV, T] extends org.apache.flink.graph.EdgesFunction[K, EV, T] {
+
+  def iterateEdges(edges: Iterable[(K, Edge[K, EV])], out: Collector[T])
+
+  override def iterateEdges(edges: java.lang.Iterable[Tuple2[K, Edge[K, EV]]], out:
+  Collector[T]): Unit = {
+    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(edges)
+      .map(jtuple => (jtuple.f0, jtuple.f1))
+    iterateEdges(scalaIterable, out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
new file mode 100644
index 0000000..82589b6
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/EdgesFunctionWithVertexValue.scala
@@ -0,0 +1,33 @@
+/*
+ * 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.scala
+
+import org.apache.flink.graph.{Edge, Vertex}
+import org.apache.flink.util.Collector
+
+abstract class EdgesFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph
+.EdgesFunctionWithVertexValue[K, VV, EV, T] {
+  @throws(classOf[Exception])
+  def iterateEdges(v: Vertex[K, VV], edges: Iterable[Edge[K, EV]], out: Collector[T])
+
+  override def iterateEdges(v: Vertex[K, VV], edges: java.lang.Iterable[Edge[K, EV]], out:
+  Collector[T]) = {
+    iterateEdges(v, scala.collection.JavaConversions.iterableAsScalaIterable(edges), out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
new file mode 100644
index 0000000..28f3f12
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala
@@ -0,0 +1,1014 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.functions.{FilterFunction, MapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.{tuple => jtuple}
+import org.apache.flink.api.scala._
+import org.apache.flink.graph._
+import org.apache.flink.graph.validation.GraphValidator
+import org.apache.flink.graph.gsa.{ApplyFunction, GSAConfiguration, GatherFunction, SumFunction}
+import org.apache.flink.graph.spargel.{MessagingFunction, VertexCentricConfiguration, VertexUpdateFunction}
+import org.apache.flink.{graph => jg}
+import _root_.scala.collection.JavaConverters._
+import _root_.scala.reflect.ClassTag
+import org.apache.flink.types.NullValue
+
+object Graph {
+
+  /**
+  * Creates a Graph from a DataSet of vertices and a DataSet of edges.
+  */
+  def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](vertices: DataSet[Vertex[K, VV]], edges: DataSet[Edge[K, EV]],
+                              env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](vertices.javaSet, edges.javaSet, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a DataSet of edges.
+  * Vertices are created automatically and their values are set to NullValue.
+  */
+  def fromDataSet[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
+  (edges: DataSet[Edge[K, EV]], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
+    wrapGraph(jg.Graph.fromDataSet[K, EV](edges.javaSet, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a graph from a DataSet of edges.
+  * Vertices are created automatically and their values are set by applying the provided
+  * map function to the vertex ids.
+  */
+  def fromDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](edges: DataSet[Edge[K, EV]], mapper: MapFunction[K, VV],
+      env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromDataSet[K, VV, EV](edges.javaSet, mapper, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a Seq of vertices and a Seq of edges.
+  */
+  def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](vertices: Seq[Vertex[K, VV]], edges: Seq[Edge[K, EV]], env:
+  ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromCollection[K, VV, EV](vertices.asJavaCollection, edges
+      .asJavaCollection, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a Seq of edges.
+  * Vertices are created automatically and their values are set to NullValue.
+  */
+  def fromCollection[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
+  (edges: Seq[Edge[K, EV]], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
+    wrapGraph(jg.Graph.fromCollection[K, EV](edges.asJavaCollection, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a graph from a Seq of edges.
+  * Vertices are created automatically and their values are set by applying the provided
+  * map function to the vertex ids.
+  */
+  def fromCollection[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](edges: Seq[Edge[K, EV]], mapper: MapFunction[K, VV],
+      env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    wrapGraph(jg.Graph.fromCollection[K, VV, EV](edges.asJavaCollection, mapper, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a DataSets of Tuples.
+  */
+  def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](vertices: DataSet[(K, VV)], edges: DataSet[(K, K, EV)],
+                              env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    val javaTupleVertices = vertices.map(v => new jtuple.Tuple2(v._1, v._2)).javaSet
+    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
+    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleVertices, javaTupleEdges,
+        env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a DataSet of Tuples representing the edges.
+  * Vertices are created automatically and their values are set to NullValue.
+  */
+  def fromTupleDataSet[K: TypeInformation : ClassTag, EV: TypeInformation : ClassTag]
+  (edges: DataSet[(K, K, EV)], env: ExecutionEnvironment): Graph[K, NullValue, EV] = {
+    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
+    wrapGraph(jg.Graph.fromTupleDataSet[K, EV](javaTupleEdges, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph from a DataSet of Tuples representing the edges.
+  * Vertices are created automatically and their values are set by applying the provided
+  * map function to the vertex ids.
+  */
+  def fromTupleDataSet[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+  TypeInformation : ClassTag](edges: DataSet[(K, K, EV)], mapper: MapFunction[K, VV],
+      env: ExecutionEnvironment): Graph[K, VV, EV] = {
+    val javaTupleEdges = edges.map(v => new jtuple.Tuple3(v._1, v._2, v._3)).javaSet
+    wrapGraph(jg.Graph.fromTupleDataSet[K, VV, EV](javaTupleEdges, mapper, env.getJavaEnv))
+  }
+
+  /**
+  * Creates a Graph with from a CSV file of vertices and a CSV file of edges
+  * 
+  * @param The Execution Environment.
+  * @param pathEdges The file path containing the edges.
+  * @param readVertices Defines whether the vertices have associated values.
+  * If set to false, the vertex input is ignored and vertices are created from the edges file.
+  * @param pathVertices The file path containing the vertices.
+  * @param hasEdgeValues Defines whether the edges have associated values. True by default.
+  * @param lineDelimiterVertices The string that separates lines in the vertices file.
+  * It defaults to newline.
+  * @param fieldDelimiterVertices The string that separates vertex Ids from vertex values
+  * in the vertices file.
+  * @param quoteCharacterVertices The character to use for quoted String parsing
+  * in the vertices file. Disabled by default.
+  * @param ignoreFirstLineVertices Whether the first line in the vertices file should be ignored.
+  * @param ignoreCommentsVertices Lines that start with the given String in the vertices file
+  * are ignored, disabled by default.
+  * @param lenientVertices Whether the parser should silently ignore malformed lines in the
+  * vertices file.
+  * @param includedFieldsVertices The fields in the vertices file that should be read.
+  * By default all fields are read.
+  * @param lineDelimiterEdges The string that separates lines in the edges file.
+  * It defaults to newline.
+  * @param fieldDelimiterEdges The string that separates fields in the edges file.
+  * @param quoteCharacterEdges The character to use for quoted String parsing
+  * in the edges file. Disabled by default.
+  * @param ignoreFirstLineEdges Whether the first line in the vertices file should be ignored.
+  * @param ignoreCommentsEdges Lines that start with the given String in the edges file
+  * are ignored, disabled by default.
+  * @param lenientEdges Whether the parser should silently ignore malformed lines in the
+  * edges file.
+  * @param includedFieldsEdges The fields in the edges file that should be read.
+  * By default all fields are read.
+  * @param mapper If no vertex values are provided, this mapper can be used to initialize them.
+  * 
+  */
+  // scalastyle:off
+  // This method exceeds the max allowed number of parameters -->  
+  def fromCsvReader[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag,
+    EV: TypeInformation : ClassTag](
+      env: ExecutionEnvironment,
+      pathEdges: String,
+      readVertices: Boolean,
+      pathVertices: String = null,
+      hasEdgeValues: Boolean = true,
+      lineDelimiterVertices: String = "\n",
+      fieldDelimiterVertices: String = ",",
+      quoteCharacterVertices: Character = null,
+      ignoreFirstLineVertices: Boolean = false,
+      ignoreCommentsVertices: String = null,
+      lenientVertices: Boolean = false,
+      includedFieldsVertices: Array[Int] = null,
+      lineDelimiterEdges: String = "\n",
+      fieldDelimiterEdges: String = ",",
+      quoteCharacterEdges: Character = null,
+      ignoreFirstLineEdges: Boolean = false,
+      ignoreCommentsEdges: String = null,
+      lenientEdges: Boolean = false,
+      includedFieldsEdges: Array[Int] = null,
+      mapper: MapFunction[K, VV] = null) = {
+
+    // with vertex and edge values
+    if (readVertices && hasEdgeValues) {
+      if (pathVertices.equals(null)) {
+        throw new IllegalArgumentException(
+            "The vertices file path must be specified when readVertices is true.")
+      } else {
+        val vertices = env.readCsvFile[(K, VV)](pathVertices, lineDelimiterVertices,
+            fieldDelimiterVertices, quoteCharacterVertices, ignoreFirstLineVertices,
+            ignoreCommentsVertices, lenientVertices, includedFieldsVertices)
+
+        val edges = env.readCsvFile[(K, K, EV)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
+            quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
+            includedFieldsEdges)
+     
+        fromTupleDataSet[K, VV, EV](vertices, edges, env) 
+      }
+    }
+    // with vertex value and no edge value
+    else if (readVertices && (!hasEdgeValues)) {
+       if (pathVertices.equals(null)) {
+        throw new IllegalArgumentException(
+            "The vertices file path must be specified when readVertices is true.")
+      } else {
+        val vertices = env.readCsvFile[(K, VV)](pathVertices, lineDelimiterVertices,
+            fieldDelimiterVertices, quoteCharacterVertices, ignoreFirstLineVertices,
+            ignoreCommentsVertices, lenientVertices, includedFieldsVertices)
+
+        val edges = env.readCsvFile[(K, K)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
+            quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
+            includedFieldsEdges).map(edge => (edge._1, edge._2, NullValue.getInstance))
+
+        fromTupleDataSet[K, VV, NullValue](vertices, edges, env)
+      }
+    }
+    // with edge value and no vertex value
+    else if ((!readVertices) && hasEdgeValues) {
+      val edges = env.readCsvFile[(K, K, EV)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
+        quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges, lenientEdges,
+        includedFieldsEdges)
+
+      // initializer provided
+      if (mapper != null) {
+        fromTupleDataSet[K, VV, EV](edges, mapper, env)
+      }
+      else {
+        fromTupleDataSet[K, EV](edges, env) 
+      }
+    }
+    // with no edge value and no vertex value
+    else {
+      val edges = env.readCsvFile[(K, K)](pathEdges, lineDelimiterEdges, fieldDelimiterEdges,
+      quoteCharacterEdges, ignoreFirstLineEdges, ignoreCommentsEdges,
+      lenientEdges, includedFieldsEdges).map(edge => (edge._1, edge._2, NullValue.getInstance))
+
+      // no initializer provided
+      if (mapper != null) {
+        fromTupleDataSet[K, VV, NullValue](edges, mapper, env)
+      }
+      else {
+        fromTupleDataSet[K, NullValue](edges, env) 
+      }
+    }
+  }
+// scalastyle:on
+
+}
+
+/**
+ * Represents a graph consisting of {@link Edge edges} and {@link Vertex vertices}.
+ * @param jgraph the underlying java api Graph.
+ * @tparam K the key type for vertex and edge identifiers
+ * @tparam VV the value type for vertices
+ * @tparam EV the value type for edges
+ * @see org.apache.flink.graph.Edge
+ * @see org.apache.flink.graph.Vertex
+ */
+final class Graph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag, EV:
+TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) {
+
+  private[flink] def getWrappedGraph = jgraph
+
+
+  private[flink] def clean[F <: AnyRef](f: F, checkSerializable: Boolean = true): F = {
+    if (jgraph.getContext.getConfig.isClosureCleanerEnabled) {
+      ClosureCleaner.clean(f, checkSerializable)
+    }
+    ClosureCleaner.ensureSerializable(f)
+    f
+  }
+
+  /**
+   * @return the vertex DataSet.
+   */
+  def getVertices = wrap(jgraph.getVertices)
+
+  /**
+   * @return the edge DataSet.
+   */
+  def getEdges = wrap(jgraph.getEdges)
+
+  /**
+   * @return the vertex DataSet as Tuple2.
+   */
+  def getVerticesAsTuple2(): DataSet[(K, VV)] = {
+    wrap(jgraph.getVerticesAsTuple2).map(jtuple => (jtuple.f0, jtuple.f1))
+  }
+
+  /**
+   * @return the edge DataSet as Tuple3.
+   */
+  def getEdgesAsTuple3(): DataSet[(K, K, EV)] = {
+    wrap(jgraph.getEdgesAsTuple3).map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2))
+  }
+
+  /**
+  * @return a DataSet of Triplets,
+  * consisting of (srcVertexId, trgVertexId, srcVertexValue, trgVertexValue, edgeValue)
+  */
+  def getTriplets(): DataSet[Triplet[K, VV, EV]] = {
+    wrap(jgraph.getTriplets())
+  }
+
+  /**
+   * Apply a function to the attribute of each vertex in the graph.
+   *
+   * @param mapper the map function to apply.
+   * @return a new graph
+   */
+  def mapVertices[NV: TypeInformation : ClassTag](mapper: MapFunction[Vertex[K, VV], NV]):
+  Graph[K, NV, EV] = {
+    new Graph[K, NV, EV](jgraph.mapVertices[NV](
+      mapper,
+      createTypeInformation[Vertex[K, NV]]
+    ))
+  }
+
+  /**
+   * Apply a function to the attribute of each vertex in the graph.
+   *
+   * @param fun the map function to apply.
+   * @return a new graph
+   */
+  def mapVertices[NV: TypeInformation : ClassTag](fun: Vertex[K, VV] => NV): Graph[K, NV, EV] = {
+    val mapper: MapFunction[Vertex[K, VV], NV] = new MapFunction[Vertex[K, VV], NV] {
+      val cleanFun = clean(fun)
+
+      def map(in: Vertex[K, VV]): NV = cleanFun(in)
+    }
+    new Graph[K, NV, EV](jgraph.mapVertices[NV](mapper, createTypeInformation[Vertex[K, NV]]))
+  }
+
+  /**
+   * Apply a function to the attribute of each edge in the graph.
+   *
+   * @param mapper the map function to apply.
+   * @return a new graph
+   */
+  def mapEdges[NV: TypeInformation : ClassTag](mapper: MapFunction[Edge[K, EV], NV]): Graph[K,
+    VV, NV] = {
+    new Graph[K, VV, NV](jgraph.mapEdges[NV](
+      mapper,
+      createTypeInformation[Edge[K, NV]]
+    ))
+  }
+
+  /**
+   * Apply a function to the attribute of each edge in the graph.
+   *
+   * @param fun the map function to apply.
+   * @return a new graph
+   */
+  def mapEdges[NV: TypeInformation : ClassTag](fun: Edge[K, EV] => NV): Graph[K, VV, NV] = {
+    val mapper: MapFunction[Edge[K, EV], NV] = new MapFunction[Edge[K, EV], NV] {
+      val cleanFun = clean(fun)
+
+      def map(in: Edge[K, EV]): NV = cleanFun(in)
+    }
+    new Graph[K, VV, NV](jgraph.mapEdges[NV](mapper, createTypeInformation[Edge[K, NV]]))
+  }
+
+  /**
+   * Joins the vertex DataSet of this graph with an input DataSet and applies
+   * a UDF on the resulted values.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param mapper the UDF map function to apply.
+   * @return a new graph where the vertex values have been updated.
+   */
+  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper: MapFunction[
+    (VV, T), VV]): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
+      override def map(value: jtuple.Tuple2[VV, T]): VV = {
+        mapper.map((value.f0, value.f1))
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the vertex DataSet of this graph with an input DataSet and applies
+   * a UDF on the resulted values.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param fun the UDF map function to apply.
+   * @return a new graph where the vertex values have been updated.
+   */
+  def joinWithVertices[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (VV, T) => VV):
+  Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[VV, T], VV]() {
+      val cleanFun = clean(fun)
+
+      override def map(value: jtuple.Tuple2[VV, T]): VV = {
+        cleanFun(value.f0, value.f1)
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithVertices[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on a composite key of both
+   * source and target and applies a UDF on the resulted values.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param mapper the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], mapper: MapFunction[
+    (EV, T), EV]): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        mapper.map((value.f0, value.f1))
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
+      scalatuple._2, scalatuple._3)).javaSet
+    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on a composite key of both
+   * source and target and applies a UDF on the resulted values.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param fun the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdges[T: TypeInformation](inputDataSet: DataSet[(K, K, T)], fun: (EV, T) => EV):
+  Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      val cleanFun = clean(fun)
+
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        cleanFun(value.f0, value.f1)
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple3(scalatuple._1,
+      scalatuple._2, scalatuple._3)).javaSet
+    wrapGraph(jgraph.joinWithEdges[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on the source key of the
+   * edges and the first attribute of the input DataSet and applies a UDF on
+   * the resulted values. In case the inputDataSet contains the same key more
+   * than once, only the first value will be considered.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param mapper the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
+  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        mapper.map((value.f0, value.f1))
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on the source key of the
+   * edges and the first attribute of the input DataSet and applies a UDF on
+   * the resulted values. In case the inputDataSet contains the same key more
+   * than once, only the first value will be considered.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param fun the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdgesOnSource[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
+    EV): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      val cleanFun = clean(fun)
+
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        cleanFun(value.f0, value.f1)
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithEdgesOnSource[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on the target key of the
+   * edges and the first attribute of the input DataSet and applies a UDF on
+   * the resulted values. Should the inputDataSet contain the same key more
+   * than once, only the first value will be considered.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param mapper the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], mapper:
+  MapFunction[(EV, T), EV]): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        mapper.map((value.f0, value.f1))
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Joins the edge DataSet with an input DataSet on the target key of the
+   * edges and the first attribute of the input DataSet and applies a UDF on
+   * the resulted values. Should the inputDataSet contain the same key more
+   * than once, only the first value will be considered.
+   *
+   * @param inputDataSet the DataSet to join with.
+   * @param fun the UDF map function to apply.
+   * @tparam T the return type
+   * @return a new graph where the edge values have been updated.
+   */
+  def joinWithEdgesOnTarget[T: TypeInformation](inputDataSet: DataSet[(K, T)], fun: (EV, T) =>
+    EV): Graph[K, VV, EV] = {
+    val newmapper = new MapFunction[jtuple.Tuple2[EV, T], EV]() {
+      val cleanFun = clean(fun)
+
+      override def map(value: jtuple.Tuple2[EV, T]): EV = {
+        cleanFun(value.f0, value.f1)
+      }
+    }
+    val javaTupleSet = inputDataSet.map(scalatuple => new jtuple.Tuple2(scalatuple._1,
+      scalatuple._2)).javaSet
+    wrapGraph(jgraph.joinWithEdgesOnTarget[T](javaTupleSet, newmapper))
+  }
+
+  /**
+   * Apply filtering functions to the graph and return a sub-graph that
+   * satisfies the predicates for both vertices and edges.
+   *
+   * @param vertexFilter the filter function for vertices.
+   * @param edgeFilter the filter function for edges.
+   * @return the resulting sub-graph.
+   */
+  def subgraph(vertexFilter: FilterFunction[Vertex[K, VV]], edgeFilter: FilterFunction[Edge[K,
+    EV]]) = {
+    wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter))
+  }
+
+  /**
+   * Apply filtering functions to the graph and return a sub-graph that
+   * satisfies the predicates for both vertices and edges.
+   *
+   * @param vertexFilterFun the filter function for vertices.
+   * @param edgeFilterFun the filter function for edges.
+   * @return the resulting sub-graph.
+   */
+  def subgraph(vertexFilterFun: Vertex[K, VV] => Boolean, edgeFilterFun: Edge[K, EV] =>
+    Boolean) = {
+    val vertexFilter = new FilterFunction[Vertex[K, VV]] {
+      val cleanVertexFun = clean(vertexFilterFun)
+
+      override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value)
+    }
+
+    val edgeFilter = new FilterFunction[Edge[K, EV]] {
+      val cleanEdgeFun = clean(edgeFilterFun)
+
+      override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value)
+    }
+
+    wrapGraph(jgraph.subgraph(vertexFilter, edgeFilter))
+  }
+
+  /**
+   * Apply a filtering function to the graph and return a sub-graph that
+   * satisfies the predicates only for the vertices.
+   *
+   * @param vertexFilter the filter function for vertices.
+   * @return the resulting sub-graph.
+   */
+  def filterOnVertices(vertexFilter: FilterFunction[Vertex[K, VV]]) = {
+    wrapGraph(jgraph.filterOnVertices(vertexFilter))
+  }
+
+  /**
+   * Apply a filtering function to the graph and return a sub-graph that
+   * satisfies the predicates only for the vertices.
+   *
+   * @param vertexFilterFun the filter function for vertices.
+   * @return the resulting sub-graph.
+   */
+  def filterOnVertices(vertexFilterFun: Vertex[K, VV] => Boolean) = {
+    val vertexFilter = new FilterFunction[Vertex[K, VV]] {
+      val cleanVertexFun = clean(vertexFilterFun)
+
+      override def filter(value: Vertex[K, VV]): Boolean = cleanVertexFun(value)
+    }
+
+    wrapGraph(jgraph.filterOnVertices(vertexFilter))
+  }
+
+  /**
+   * Apply a filtering function to the graph and return a sub-graph that
+   * satisfies the predicates only for the edges.
+   *
+   * @param edgeFilter the filter function for edges.
+   * @return the resulting sub-graph.
+   */
+  def filterOnEdges(edgeFilter: FilterFunction[Edge[K, EV]]) = {
+    wrapGraph(jgraph.filterOnEdges(edgeFilter))
+  }
+
+  /**
+   * Apply a filtering function to the graph and return a sub-graph that
+   * satisfies the predicates only for the edges.
+   *
+   * @param edgeFilterFun the filter function for edges.
+   * @return the resulting sub-graph.
+   */
+  def filterOnEdges(edgeFilterFun: Edge[K, EV] => Boolean) = {
+    val edgeFilter = new FilterFunction[Edge[K, EV]] {
+      val cleanEdgeFun = clean(edgeFilterFun)
+
+      override def filter(value: Edge[K, EV]): Boolean = cleanEdgeFun(value)
+    }
+
+    wrapGraph(jgraph.filterOnEdges(edgeFilter))
+  }
+
+  /**
+   * Return the in-degree of all vertices in the graph
+   *
+   * @return A DataSet of Tuple2<vertexId, inDegree>
+   */
+  def inDegrees(): DataSet[(K, Long)] = {
+    wrap(jgraph.inDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
+  }
+
+  /**
+   * Return the out-degree of all vertices in the graph
+   *
+   * @return A DataSet of Tuple2<vertexId, outDegree>
+   */
+  def outDegrees(): DataSet[(K, Long)] = {
+    wrap(jgraph.outDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
+  }
+
+  /**
+   * Return the degree of all vertices in the graph
+   *
+   * @return A DataSet of Tuple2<vertexId, degree>
+   */
+  def getDegrees(): DataSet[(K, Long)] = {
+    wrap(jgraph.getDegrees).map(javatuple => (javatuple.f0, javatuple.f1))
+  }
+
+  /**
+   * This operation adds all inverse-direction edges to the graph.
+   *
+   * @return the undirected graph.
+   */
+  def getUndirected(): Graph[K, VV, EV] = {
+    new Graph(jgraph.getUndirected)
+  }
+
+  /**
+   * Reverse the direction of the edges in the graph
+   *
+   * @return a new graph with all edges reversed
+   * @throws UnsupportedOperationException
+   */
+  def reverse(): Graph[K, VV, EV] = {
+    new Graph(jgraph.reverse())
+  }
+
+  /**
+   * Compute an aggregate over the edges of each vertex. The function applied
+   * on the edges has access to the vertex value.
+   *
+   * @param edgesFunction the function to apply to the neighborhood
+   * @param direction     the edge direction (in-, out-, all-)
+   * @tparam T           the output type
+   * @return a dataset of a T
+   */
+  def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction:
+                                                        EdgesFunctionWithVertexValue[K, VV, EV,
+                                                          T], direction: EdgeDirection):
+  DataSet[T] = {
+    wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T]))
+  }
+
+  /**
+   * Compute an aggregate over the edges of each vertex. The function applied
+   * on the edges has access to the vertex value.
+   *
+   * @param edgesFunction the function to apply to the neighborhood
+   * @param direction     the edge direction (in-, out-, all-)
+   * @tparam T           the output type
+   * @return a dataset of a T
+   */
+  def groupReduceOnEdges[T: TypeInformation : ClassTag](edgesFunction: EdgesFunction[K, EV, T],
+                                                        direction: EdgeDirection): DataSet[T] = {
+    wrap(jgraph.groupReduceOnEdges(edgesFunction, direction, createTypeInformation[T]))
+  }
+
+  /**
+   * Compute an aggregate over the neighbors (edges and vertices) of each
+   * vertex. The function applied on the neighbors has access to the vertex
+   * value.
+   *
+   * @param neighborsFunction the function to apply to the neighborhood
+   * @param direction         the edge direction (in-, out-, all-)
+   * @tparam T               the output type
+   * @return a dataset of a T
+   */
+  def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction:
+                                                            NeighborsFunctionWithVertexValue[K,
+                                                              VV, EV, T], direction:
+                                                            EdgeDirection): DataSet[T] = {
+    wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T]))
+  }
+
+  /**
+   * Compute an aggregate over the neighbors (edges and vertices) of each
+   * vertex.
+   *
+   * @param neighborsFunction the function to apply to the neighborhood
+   * @param direction         the edge direction (in-, out-, all-)
+   * @tparam T               the output type
+   * @return a dataset of a T
+   */
+  def groupReduceOnNeighbors[T: TypeInformation : ClassTag](neighborsFunction:
+                                                            NeighborsFunction[K, VV, EV, T],
+                                                            direction: EdgeDirection):
+  DataSet[T] = {
+    wrap(jgraph.groupReduceOnNeighbors(neighborsFunction, direction, createTypeInformation[T]))
+  }
+
+  /**
+   * @return a long integer representing the number of vertices
+   */
+  def numberOfVertices(): Long = {
+    jgraph.numberOfVertices()
+  }
+
+  /**
+   * @return a long integer representing the number of edges
+   */
+  def numberOfEdges(): Long = {
+    jgraph.numberOfEdges()
+  }
+
+  /**
+   * @return The IDs of the vertices as DataSet
+   */
+  def getVertexIds(): DataSet[K] = {
+    wrap(jgraph.getVertexIds)
+  }
+
+  /**
+   * @return The IDs of the edges as DataSet
+   */
+  def getEdgeIds(): DataSet[(K, K)] = {
+    wrap(jgraph.getEdgeIds).map(jtuple => (jtuple.f0, jtuple.f1))
+  }
+
+  /**
+   * Adds the input vertex to the graph. If the vertex already
+   * exists in the graph, it will not be added again.
+   *
+   * @param vertex the vertex to be added
+   * @return the new graph containing the existing vertices as well as the one just added
+   */
+  def addVertex(vertex: Vertex[K, VV]) = {
+    wrapGraph(jgraph.addVertex(vertex))
+  }
+
+  /**
+  * Adds the list of vertices, passed as input, to the graph.
+  * If the vertices already exist in the graph, they will not be added once more.
+  *
+  * @param verticesToAdd the list of vertices to add
+  * @return the new graph containing the existing and newly added vertices
+  */
+  def addVertices(vertices: List[Vertex[K, VV]]): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.addVertices(vertices.asJava))
+  }
+
+  /**
+  * Adds the given list edges to the graph.
+  *
+  * When adding an edge for a non-existing set of vertices,
+  * the edge is considered invalid and ignored.
+  *
+  * @param newEdges the data set of edges to be added
+  * @return a new graph containing the existing edges plus the newly added edges.
+  */
+  def addEdges(edges: List[Edge[K, EV]]): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.addEdges(edges.asJava))
+  }
+
+    /**
+   * Adds the given edge to the graph. If the source and target vertices do
+   * not exist in the graph, they will also be added.
+   *
+   * @param source the source vertex of the edge
+   * @param target the target vertex of the edge
+   * @param edgeValue the edge value
+   * @return the new graph containing the existing vertices and edges plus the
+   *         newly added edge
+   */
+  def addEdge(source: Vertex[K, VV], target: Vertex[K, VV], edgeValue: EV) = {
+    wrapGraph(jgraph.addEdge(source, target, edgeValue))
+  }
+
+  /**
+   * Removes the given vertex and its edges from the graph.
+   *
+   * @param vertex the vertex to remove
+   * @return the new graph containing the existing vertices and edges without
+   *         the removed vertex and its edges
+   */
+  def removeVertex(vertex: Vertex[K, VV]) = {
+    wrapGraph(jgraph.removeVertex(vertex))
+  }
+
+    /**
+   * Removes the given vertex and its edges from the graph.
+   *
+   * @param vertex the vertex to remove
+   * @return the new graph containing the existing vertices and edges without
+   *         the removed vertex and its edges
+   */
+  def removeVertices(vertices: List[Vertex[K, VV]]): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.removeVertices(vertices.asJava))
+  }
+
+  /**
+   * Removes all edges that match the given edge from the graph.
+   *
+   * @param edge the edge to remove
+   * @return the new graph containing the existing vertices and edges without
+   *         the removed edges
+   */
+  def removeEdge(edge: Edge[K, EV]) = {
+    wrapGraph(jgraph.removeEdge(edge))
+  }
+
+  /**
+   * Removes all the edges that match the edges in the given data set from the graph.
+   *
+   * @param edgesToBeRemoved the list of edges to be removed
+   * @return a new graph where the edges have been removed and in which the vertices remained intact
+   */
+  def removeEdges(edges: List[Edge[K, EV]]): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.removeEdges(edges.asJava))
+  }
+
+  /**
+   * Performs union on the vertices and edges sets of the input graphs
+   * removing duplicate vertices but maintaining duplicate edges.
+   *
+   * @param graph the graph to perform union with
+   * @return a new graph
+   */
+  def union(graph: Graph[K, VV, EV]) = {
+    wrapGraph(jgraph.union(graph.getWrappedGraph))
+  }
+
+  /**
+  * Performs Difference on the vertex and edge sets of the input graphs
+  * removes common vertices and edges. If a source/target vertex is removed,
+  * its corresponding edge will also be removed
+  * @param graph the graph to perform difference with
+  * @return a new graph where the common vertices and edges have been removed
+  */
+  def difference(graph: Graph[K, VV, EV]) = {
+    wrapGraph(jgraph.difference(graph.getWrappedGraph))
+  }
+
+  /**
+   * Compute an aggregate over the neighbor values of each
+   * vertex.
+   *
+   * @param reduceNeighborsFunction the function to apply to the neighborhood
+   * @param direction               the edge direction (in-, out-, all-)
+   * @return a Dataset containing one value per vertex (vertex id, aggregate vertex value)
+   */
+  def reduceOnNeighbors(reduceNeighborsFunction: ReduceNeighborsFunction[VV], direction:
+  EdgeDirection): DataSet[(K, VV)] = {
+    wrap(jgraph.reduceOnNeighbors(reduceNeighborsFunction, direction)).map(jtuple => (jtuple
+      .f0, jtuple.f1))
+  }
+
+  /**
+   * Compute an aggregate over the edge values of each vertex.
+   *
+   * @param reduceEdgesFunction the function to apply to the neighborhood
+   * @param direction           the edge direction (in-, out-, all-)
+   * @return a Dataset containing one value per vertex(vertex key, aggegate edge value)
+   * @throws IllegalArgumentException
+   */
+  def reduceOnEdges(reduceEdgesFunction: ReduceEdgesFunction[EV], direction: EdgeDirection):
+  DataSet[(K, EV)] = {
+    wrap(jgraph.reduceOnEdges(reduceEdgesFunction, direction)).map(jtuple => (jtuple.f0,
+      jtuple.f1))
+  }
+
+  def run[T: TypeInformation : ClassTag](algorithm: GraphAlgorithm[K, VV, EV, T]):
+  T = {
+    jgraph.run(algorithm)
+  }
+
+  /**
+   * Runs a Vertex-Centric iteration on the graph.
+   * No configuration options are provided.
+   *
+   * @param vertexUpdateFunction the vertex update function
+   * @param messagingFunction the messaging function
+   * @param maxIterations maximum number of iterations to perform
+   *
+   * @return the updated Graph after the vertex-centric iteration has converged or
+   *         after maximumNumberOfIterations.
+   */
+  def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M],
+                                   messagingFunction: MessagingFunction[K, VV, M, EV],
+                                   maxIterations: Int): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
+      maxIterations))
+  }
+
+  /**
+   * Runs a Vertex-Centric iteration on the graph with configuration options.
+   *
+   * @param vertexUpdateFunction the vertex update function
+   * @param messagingFunction the messaging function
+   * @param maxIterations maximum number of iterations to perform
+   * @param parameters the iteration configuration parameters
+   *
+   * @return the updated Graph after the vertex-centric iteration has converged or
+   *         after maximumNumberOfIterations.
+   */
+  def runVertexCentricIteration[M](vertexUpdateFunction: VertexUpdateFunction[K, VV, M],
+                                   messagingFunction: MessagingFunction[K, VV, M, EV],
+                                   maxIterations: Int, parameters: VertexCentricConfiguration):
+  Graph[K, VV, EV] = {
+    wrapGraph(jgraph.runVertexCentricIteration(vertexUpdateFunction, messagingFunction,
+      maxIterations, parameters))
+  }
+
+  /**
+   * Runs a Gather-Sum-Apply iteration on the graph.
+   * No configuration options are provided.
+   *
+   * @param gatherFunction the gather function collects information about adjacent
+   *                       vertices and edges
+   * @param sumFunction the sum function aggregates the gathered information
+   * @param applyFunction the apply function updates the vertex values with the aggregates
+   * @param maxIterations maximum number of iterations to perform
+   * @tparam M the intermediate type used between gather, sum and apply
+   *
+   * @return the updated Graph after the gather-sum-apply iteration has converged or
+   *         after maximumNumberOfIterations.
+   */
+  def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction:
+  SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int): Graph[K,
+    VV, EV] = {
+    wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
+      maxIterations))
+  }
+
+  /**
+   * Runs a Gather-Sum-Apply iteration on the graph with configuration options.
+   *
+   * @param gatherFunction the gather function collects information about adjacent
+   *                       vertices and edges
+   * @param sumFunction the sum function aggregates the gathered information
+   * @param applyFunction the apply function updates the vertex values with the aggregates
+   * @param maxIterations maximum number of iterations to perform
+   * @param parameters the iteration configuration parameters
+   * @tparam M the intermediate type used between gather, sum and apply
+   *
+   * @return the updated Graph after the gather-sum-apply iteration has converged or
+   *         after maximumNumberOfIterations.
+   */
+  def runGatherSumApplyIteration[M](gatherFunction: GatherFunction[VV, EV, M], sumFunction:
+  SumFunction[VV, EV, M], applyFunction: ApplyFunction[K, VV, M], maxIterations: Int,
+                                    parameters: GSAConfiguration): Graph[K, VV, EV] = {
+    wrapGraph(jgraph.runGatherSumApplyIteration(gatherFunction, sumFunction, applyFunction,
+      maxIterations, parameters))
+  }
+
+  def validate(validator: GraphValidator[K, VV, EV]): Boolean = {
+    jgraph.validate(validator)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
new file mode 100644
index 0000000..ca15dab
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.java.tuple.Tuple3
+import org.apache.flink.graph.{Edge, Vertex}
+import org.apache.flink.util.Collector
+
+
+abstract class NeighborsFunction[K, VV, EV, T] extends org.apache.flink.graph
+.NeighborsFunction[K, VV, EV, T] {
+
+  def iterateNeighbors(neighbors: Iterable[(K, Edge[K, EV], Vertex[K, VV])], out: Collector[T])
+
+  override def iterateNeighbors(neighbors: java.lang.Iterable[Tuple3[K, Edge[K, EV], Vertex[K,
+    VV]]], out: Collector[T]) = {
+    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors)
+      .map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2))
+    iterateNeighbors(scalaIterable, out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
new file mode 100644
index 0000000..cefc277
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
@@ -0,0 +1,40 @@
+/*
+ * 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.scala
+
+import java.lang
+
+import org.apache.flink.api.java.tuple.Tuple2
+import org.apache.flink.graph.{Edge, Vertex}
+import org.apache.flink.util.Collector
+
+
+abstract class NeighborsFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph
+.NeighborsFunctionWithVertexValue[K, VV, EV, T] {
+
+  def iterateNeighbors(vertex: Vertex[K, VV], neighbors: Iterable[(Edge[K, EV], Vertex[K, VV])
+    ], out: Collector[T]): Unit
+
+  override def iterateNeighbors(vertex: Vertex[K, VV], neighbors: lang.Iterable[Tuple2[Edge[K,
+    EV], Vertex[K, VV]]], out: Collector[T]): Unit = {
+    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors)
+      .map(jtuple => (jtuple.f0, jtuple.f1))
+    iterateNeighbors(vertex, scalaIterable, out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
new file mode 100644
index 0000000..b3da520
--- /dev/null
+++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
@@ -0,0 +1,121 @@
+/*
+ * 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.scala.example;
+
+import org.apache.flink.api.scala._
+import org.apache.flink.graph.scala._
+import org.apache.flink.graph.Edge
+import org.apache.flink.types.NullValue
+import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.graph.library.GSAConnectedComponents
+import java.lang.Long
+
+/**
+ * This example shows how to use Gelly's library methods.
+ * You can find all available library methods in [[org.apache.flink.graph.library]]. 
+ * 
+ * In particular, this example uses the
+ * [[org.apache.flink.graph.library.ConnectedComponentsAlgorithm.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 and 1-3.
+ *
+ * Usage {{
+ *   ConnectedComponents <edge path> <result path> <number of iterations>
+ *   }}
+ * If no parameters are provided, the program is run with default data from
+ * [[org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData]]
+ */
+object ConnectedComponents {
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val edges: DataSet[Edge[Long, NullValue]] = getEdgesDataSet(env)
+    val graph = Graph.fromDataSet[Long, Long, NullValue](edges, new InitVertices, env)
+
+    val components = graph.run(new GSAConnectedComponents[Long, NullValue](maxIterations))
+
+
+    // emit result
+    if (fileOutput) {
+      components.writeAsCsv(outputPath, "\n", ",")
+      env.execute("Connected Components Example")
+    } else {
+      components.print()
+    }
+  }
+
+  private final class InitVertices extends MapFunction[Long, Long] {
+    override def map(id: Long) = {id}
+  }
+
+  // ***********************************************************************
+  // UTIL METHODS
+  // ***********************************************************************
+
+    private var fileOutput = false
+    private var edgesInputPath: String = null
+    private var outputPath: String = null
+    private var maxIterations = ConnectedComponentsDefaultData.MAX_ITERATIONS
+
+    private def parseParameters(args: Array[String]): Boolean = {
+      if(args.length > 0) {
+        if(args.length != 3) {
+          System.err.println("Usage ConnectedComponents <edge path> <output path> " +
+            "<num iterations>")
+          false
+        }
+        fileOutput = true
+        edgesInputPath = args(0)
+        outputPath = args(1)
+        maxIterations = (2).toInt
+      } 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("  See the documentation for the correct format of input files.")
+        System.out.println("Usage ConnectedComponents <edge path> <output path> " +
+          "<num iterations>");
+      }
+      true
+    }
+
+    private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, NullValue]] = {
+      if (fileOutput) {
+        env.readCsvFile[(Long, Long)](edgesInputPath,
+          lineDelimiter = "\n",
+          fieldDelimiter = "\t")
+          .map(edge => new Edge[Long, NullValue](edge._1, edge._2, NullValue.getInstance))
+      } else {
+        val edgeData = ConnectedComponentsDefaultData.DEFAULT_EDGES map {
+          case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
+        }
+        env.fromCollection(edgeData).map(
+        edge => new Edge[Long, NullValue](edge._1, edge._2, NullValue.getInstance))
+      }
+    }
+}


[12/24] flink git commit: [FLINK-2833] [gelly] create a flink-libraries module and move gelly there

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
deleted file mode 100644
index ca15dab..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunction.scala
+++ /dev/null
@@ -1,37 +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.scala
-
-import org.apache.flink.api.java.tuple.Tuple3
-import org.apache.flink.graph.{Edge, Vertex}
-import org.apache.flink.util.Collector
-
-
-abstract class NeighborsFunction[K, VV, EV, T] extends org.apache.flink.graph
-.NeighborsFunction[K, VV, EV, T] {
-
-  def iterateNeighbors(neighbors: Iterable[(K, Edge[K, EV], Vertex[K, VV])], out: Collector[T])
-
-  override def iterateNeighbors(neighbors: java.lang.Iterable[Tuple3[K, Edge[K, EV], Vertex[K,
-    VV]]], out: Collector[T]) = {
-    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors)
-      .map(jtuple => (jtuple.f0, jtuple.f1, jtuple.f2))
-    iterateNeighbors(scalaIterable, out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
deleted file mode 100644
index cefc277..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/NeighborsFunctionWithVertexValue.scala
+++ /dev/null
@@ -1,40 +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.scala
-
-import java.lang
-
-import org.apache.flink.api.java.tuple.Tuple2
-import org.apache.flink.graph.{Edge, Vertex}
-import org.apache.flink.util.Collector
-
-
-abstract class NeighborsFunctionWithVertexValue[K, VV, EV, T] extends org.apache.flink.graph
-.NeighborsFunctionWithVertexValue[K, VV, EV, T] {
-
-  def iterateNeighbors(vertex: Vertex[K, VV], neighbors: Iterable[(Edge[K, EV], Vertex[K, VV])
-    ], out: Collector[T]): Unit
-
-  override def iterateNeighbors(vertex: Vertex[K, VV], neighbors: lang.Iterable[Tuple2[Edge[K,
-    EV], Vertex[K, VV]]], out: Collector[T]): Unit = {
-    val scalaIterable = scala.collection.JavaConversions.iterableAsScalaIterable(neighbors)
-      .map(jtuple => (jtuple.f0, jtuple.f1))
-    iterateNeighbors(vertex, scalaIterable, out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
deleted file mode 100644
index b3da520..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/ConnectedComponents.scala
+++ /dev/null
@@ -1,121 +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.scala.example;
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.Edge
-import org.apache.flink.types.NullValue
-import org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.library.GSAConnectedComponents
-import java.lang.Long
-
-/**
- * This example shows how to use Gelly's library methods.
- * You can find all available library methods in [[org.apache.flink.graph.library]]. 
- * 
- * In particular, this example uses the
- * [[org.apache.flink.graph.library.ConnectedComponentsAlgorithm.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 and 1-3.
- *
- * Usage {{
- *   ConnectedComponents <edge path> <result path> <number of iterations>
- *   }}
- * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.graph.example.utils.ConnectedComponentsDefaultData]]
- */
-object ConnectedComponents {
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edges: DataSet[Edge[Long, NullValue]] = getEdgesDataSet(env)
-    val graph = Graph.fromDataSet[Long, Long, NullValue](edges, new InitVertices, env)
-
-    val components = graph.run(new GSAConnectedComponents[Long, NullValue](maxIterations))
-
-
-    // emit result
-    if (fileOutput) {
-      components.writeAsCsv(outputPath, "\n", ",")
-      env.execute("Connected Components Example")
-    } else {
-      components.print()
-    }
-  }
-
-  private final class InitVertices extends MapFunction[Long, Long] {
-    override def map(id: Long) = {id}
-  }
-
-  // ***********************************************************************
-  // UTIL METHODS
-  // ***********************************************************************
-
-    private var fileOutput = false
-    private var edgesInputPath: String = null
-    private var outputPath: String = null
-    private var maxIterations = ConnectedComponentsDefaultData.MAX_ITERATIONS
-
-    private def parseParameters(args: Array[String]): Boolean = {
-      if(args.length > 0) {
-        if(args.length != 3) {
-          System.err.println("Usage ConnectedComponents <edge path> <output path> " +
-            "<num iterations>")
-          false
-        }
-        fileOutput = true
-        edgesInputPath = args(0)
-        outputPath = args(1)
-        maxIterations = (2).toInt
-      } 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("  See the documentation for the correct format of input files.")
-        System.out.println("Usage ConnectedComponents <edge path> <output path> " +
-          "<num iterations>");
-      }
-      true
-    }
-
-    private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, NullValue]] = {
-      if (fileOutput) {
-        env.readCsvFile[(Long, Long)](edgesInputPath,
-          lineDelimiter = "\n",
-          fieldDelimiter = "\t")
-          .map(edge => new Edge[Long, NullValue](edge._1, edge._2, NullValue.getInstance))
-      } else {
-        val edgeData = ConnectedComponentsDefaultData.DEFAULT_EDGES map {
-          case Array(x, y) => (x.asInstanceOf[Long], y.asInstanceOf[Long])
-        }
-        env.fromCollection(edgeData).map(
-        edge => new Edge[Long, NullValue](edge._1, edge._2, NullValue.getInstance))
-      }
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
deleted file mode 100644
index 2dc272c..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GSASingleSourceShortestPaths.scala
+++ /dev/null
@@ -1,156 +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.scala.example;
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.types.NullValue
-import org.apache.flink.graph.Edge
-import org.apache.flink.api.common.functions.MapFunction
-import scala.collection.JavaConversions._
-import org.apache.flink.graph.scala.utils.Tuple3ToEdgeMap
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData
-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.gsa.ApplyFunction
-
-/**
- * This example shows how to use Gelly's gather-sum-apply iterations.
- * 
- * It is an implementation of the Single-Source-Shortest-Paths algorithm. 
- *
- * The input file is a plain text file and must be formatted as follows:
- * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
- * separated by tabs. Edges themselves are separated by newlines.
- * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
- * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
- *
- * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.graph.example.utils.SingleSourceShortestPathsData]]
- */
-object GSASingleSourceShortestPaths {
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edges: DataSet[Edge[Long, Double]] = getEdgesDataSet(env)
-    val graph = Graph.fromDataSet[Long, Double, Double](edges, new InitVertices(srcVertexId), env)
-
-    // Execute the gather-sum-apply iteration
-    val result = graph.runGatherSumApplyIteration(new CalculateDistances, new ChooseMinDistance,
-      new UpdateDistance, maxIterations)
-
-    // Extract the vertices as the result
-    val singleSourceShortestPaths = result.getVertices
-
-    // emit result
-    if (fileOutput) {
-      singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",")
-      env.execute("GSA Single Source Shortest Paths Example")
-    } else {
-      singleSourceShortestPaths.print()
-    }
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Single Source Shortest Path UDFs
-  // --------------------------------------------------------------------------------------------
-
-  private final class InitVertices(srcId: Long) extends MapFunction[Long, Double] {
-
-    override def map(id: Long) = {
-      if (id.equals(srcId)) {
-        0.0
-      } else {
-        Double.PositiveInfinity
-      }
-    }
-  }
-
-  private final class CalculateDistances extends GatherFunction[Double, Double, Double] {
-    override def gather(neighbor: Neighbor[Double, Double]) = {
-      neighbor.getNeighborValue + neighbor.getEdgeValue
-    }
-  }
-
-  private final class ChooseMinDistance extends SumFunction[Double, Double, Double] {
-    override def sum(newValue: Double, currentValue: Double) = {
-      Math.min(newValue, currentValue)
-    }
-  }
-
-  private final class UpdateDistance extends ApplyFunction[Long, Double, Double] {
-    override def apply(newDistance: Double, oldDistance: Double) = {
-      if (newDistance < oldDistance) {
-        setResult(newDistance)
-      }
-    }
-  }
-
-  // **************************************************************************
-  // UTIL METHODS
-  // **************************************************************************
-
-  private var fileOutput = false
-  private var srcVertexId = 1L
-  private var edgesInputPath: String = null
-  private var outputPath: String = null
-  private var maxIterations = 5
-
-  private def parseParameters(args: Array[String]): Boolean = {
-    if(args.length > 0) {
-      if(args.length != 4) {
-        System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-          " <input edges path> <output path> <num iterations>")
-        false
-      }
-      fileOutput = true
-      srcVertexId = args(0).toLong
-      edgesInputPath = args(1)
-      outputPath = args(2)
-      maxIterations = (3).toInt
-    } else {
-      System.out.println("Executing Single Source Shortest Paths example "
-        + "with default parameters and built-in default data.")
-      System.out.println("  Provide parameters to read input data from files.")
-      System.out.println("  See the documentation for the correct format of input files.")
-      System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-        " <input edges path> <output path> <num iterations>");
-    }
-    true
-  }
-
-  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, Double]] = {
-    if (fileOutput) {
-      env.readCsvFile[(Long, Long, Double)](edgesInputPath,
-        lineDelimiter = "\n",
-        fieldDelimiter = "\t")
-        .map(new Tuple3ToEdgeMap[Long, Double]())
-    } else {
-      val edgeData = SingleSourceShortestPathsData.DEFAULT_EDGES map {
-        case Array(x, y, z) => (x.asInstanceOf[Long], y.asInstanceOf[Long],
-          z.asInstanceOf[Double])
-      }
-      env.fromCollection(edgeData).map(new Tuple3ToEdgeMap[Long, Double]())
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
deleted file mode 100644
index 4eed824..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/GraphMetrics.scala
+++ /dev/null
@@ -1,128 +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.scala.example
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.types.NullValue
-import org.apache.flink.graph.Edge
-import org.apache.flink.util.Collector
-
-/**
- * This example illustrates how to use Gelly metrics methods and get simple statistics
- * from the input graph.  
- * 
- * The program creates a random graph and computes and prints
- * the following metrics:
- * - number of vertices
- * - number of edges
- * - average node degree
- * - the vertex ids with the max/min in- and out-degrees
- *
- * The input file is expected to contain one edge per line,
- * with long IDs and no values, in the following format:
- * {{{
- *   <sourceVertexID>\t<targetVertexID>
- * }}}
- * If no arguments are provided, the example runs with a random graph of 100 vertices.
- *
- */
-object GraphMetrics {
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    /** create the graph **/
-    val graph: Graph[Long, NullValue, NullValue] = Graph.fromDataSet(getEdgeDataSet(env), env)
-
-    /** get the number of vertices **/
-    val numVertices = graph.numberOfVertices;
-
-    /** get the number of edges **/
-    val numEdges = graph.numberOfEdges;
-
-    /** compute the average node degree **/
-    val verticesWithDegrees = graph.getDegrees;
-    val avgDegree = verticesWithDegrees.sum(1).map(in => (in._2 / numVertices).toDouble)
-
-    /** find the vertex with the maximum in-degree **/
-    val maxInDegreeVertex = graph.inDegrees.max(1).map(in => in._1)
-
-    /** find the vertex with the minimum in-degree **/
-    val minInDegreeVertex = graph.inDegrees.min(1).map(in => in._1)
-
-    /** find the vertex with the maximum out-degree **/
-    val maxOutDegreeVertex = graph.outDegrees.max(1).map(in => in._1)
-
-    /** find the vertex with the minimum out-degree **/
-    val minOutDegreeVertex = graph.outDegrees.min(1).map(in => in._1)
-
-    /** print the results **/
-    env.fromElements(numVertices).printOnTaskManager("Total number of vertices")
-    env.fromElements(numEdges).printOnTaskManager("Total number of edges")
-    avgDegree.printOnTaskManager("Average node degree")
-    maxInDegreeVertex.printOnTaskManager("Vertex with Max in-degree")
-    minInDegreeVertex.printOnTaskManager("Vertex with Max in-degree")
-    maxOutDegreeVertex.printOnTaskManager("Vertex with Max out-degree")
-    minOutDegreeVertex.printOnTaskManager("Vertex with Max out-degree")
-
-  }
-
-  private def parseParameters(args: Array[String]): Boolean = {
-    if (args.length > 0) {
-      fileOutput = true
-      if (args.length == 1) {
-        edgesPath = args(0)
-        true
-      } else {
-        System.err.println("Usage: GraphMetrics <edges path>")
-        false
-      }
-    } else {
-      System.out.println("Executing GraphMetrics example with built-in default data.")
-      System.out.println("  Provide parameters to read input data from a file.")
-      System.out.println("  Usage: GraphMetrics <edges path>")
-      true
-    }
-  }
-
-  private def getEdgeDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, NullValue]] = {
-    if (fileOutput) {
-      env.readCsvFile[(Long, Long)](
-        edgesPath,
-        fieldDelimiter = "\t").map(
-        in => new Edge[Long, NullValue](in._1, in._2, NullValue.getInstance()))
-    } else {
-      env.generateSequence(1, numVertices).flatMap[Edge[Long, NullValue]](
-        (key: Long, out: Collector[Edge[Long, NullValue]]) => {
-          val numOutEdges: Int = (Math.random() * (numVertices / 2)).toInt
-          for ( i <- 0 to numOutEdges ) {
-            var target: Long = ((Math.random() * numVertices) + 1).toLong
-            new Edge[Long, NullValue](key, target, NullValue.getInstance())
-          }
-      })
-    }
-  }
-
-  private var fileOutput: Boolean = false
-  private var edgesPath: String = null
-  private var outputPath: String = null
-  private val numVertices = 100
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
deleted file mode 100644
index 65a8e7f..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/example/SingleSourceShortestPaths.scala
+++ /dev/null
@@ -1,170 +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.scala.example;
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.types.NullValue
-import org.apache.flink.graph.Edge
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.spargel.VertexUpdateFunction
-import org.apache.flink.graph.spargel.MessageIterator
-import org.apache.flink.graph.Vertex
-import org.apache.flink.graph.spargel.MessagingFunction
-import scala.collection.JavaConversions._
-import org.apache.flink.graph.scala.utils.Tuple3ToEdgeMap
-import org.apache.flink.graph.example.utils.SingleSourceShortestPathsData
-
-/**
- * This example shows how to use Gelly's vertex-centric iterations.
- * 
- * It is an implementation of the Single-Source-Shortest-Paths algorithm. 
- *
- * The input file is a plain text file and must be formatted as follows:
- * Edges are represented by tuples of srcVertexId, trgVertexId, distance which are
- * separated by tabs. Edges themselves are separated by newlines.
- * For example: <code>1\t2\t0.1\n1\t3\t1.4\n</code> defines two edges,
- * edge 1-2 with distance 0.1, and edge 1-3 with distance 1.4.
- *
- * If no parameters are provided, the program is run with default data from
- * [[org.apache.flink.graph.example.utils.SingleSourceShortestPathsData]]
- */
-object SingleSourceShortestPaths {
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edges: DataSet[Edge[Long, Double]] = getEdgesDataSet(env)
-    val graph = Graph.fromDataSet[Long, Double, Double](edges, new InitVertices(srcVertexId), env)
-
-    // Execute the vertex-centric iteration
-    val result = graph.runVertexCentricIteration(new VertexDistanceUpdater,
-      new MinDistanceMessenger, maxIterations)
-
-    // Extract the vertices as the result
-    val singleSourceShortestPaths = result.getVertices
-
-    // emit result
-    if (fileOutput) {
-      singleSourceShortestPaths.writeAsCsv(outputPath, "\n", ",")
-      env.execute("Single Source Shortest Paths Example")
-    } else {
-      singleSourceShortestPaths.print()
-    }
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Single Source Shortest Path UDFs
-  // --------------------------------------------------------------------------------------------
-
-  private final class InitVertices(srcId: Long) extends MapFunction[Long, Double] {
-
-    override def map(id: Long) = {
-      if (id.equals(srcId)) {
-        0.0
-      } else {
-        Double.PositiveInfinity
-      }
-    }
-  }
-
-  /**
-   * Function that updates the value of a vertex by picking the minimum
-   * distance from all incoming messages.
-   */
-  private final class VertexDistanceUpdater extends VertexUpdateFunction[Long, Double, Double] {
-
-    override def updateVertex(vertex: Vertex[Long, Double], inMessages: MessageIterator[Double]) {
-      var minDistance = Double.MaxValue
-      while (inMessages.hasNext) {
-        var msg = inMessages.next
-        if (msg < minDistance) {
-          minDistance = msg
-        }
-      }
-      if (vertex.getValue > minDistance) {
-        setNewVertexValue(minDistance)
-      }
-    }
-  }
-
-  /**
-   * Distributes the minimum distance associated with a given vertex among all
-   * the target vertices summed up with the edge's value.
-   */
-  private final class MinDistanceMessenger extends
-    MessagingFunction[Long, Double, Double, Double] {
-
-    override def sendMessages(vertex: Vertex[Long, Double]) {
-      for (edge: Edge[Long, Double] <- getEdges) {
-        sendMessageTo(edge.getTarget(), vertex.getValue + edge.getValue)
-      }
-    }
-  }
-
-  // ****************************************************************************
-  // UTIL METHODS
-  // ****************************************************************************
-
-  private var fileOutput = false
-  private var srcVertexId = 1L
-  private var edgesInputPath: String = null
-  private var outputPath: String = null
-  private var maxIterations = 5
-
-  private def parseParameters(args: Array[String]): Boolean = {
-    if(args.length > 0) {
-      if(args.length != 4) {
-        System.err.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-          " <input edges path> <output path> <num iterations>")
-        false
-      }
-      fileOutput = true
-      srcVertexId = args(0).toLong
-      edgesInputPath = args(1)
-      outputPath = args(2)
-      maxIterations = (3).toInt
-    } else {
-      System.out.println("Executing Single Source Shortest Paths example "
-        + "with default parameters and built-in default data.")
-      System.out.println("  Provide parameters to read input data from files.")
-      System.out.println("  See the documentation for the correct format of input files.")
-      System.out.println("Usage: SingleSourceShortestPaths <source vertex id>" +
-        " <input edges path> <output path> <num iterations>");
-    }
-    true
-  }
-
-  private def getEdgesDataSet(env: ExecutionEnvironment): DataSet[Edge[Long, Double]] = {
-    if (fileOutput) {
-      env.readCsvFile[(Long, Long, Double)](edgesInputPath,
-        lineDelimiter = "\n",
-        fieldDelimiter = "\t")
-        .map(new Tuple3ToEdgeMap[Long, Double]())
-    } else {
-      val edgeData = SingleSourceShortestPathsData.DEFAULT_EDGES map {
-        case Array(x, y, z) => (x.asInstanceOf[Long], y.asInstanceOf[Long],
-          z.asInstanceOf[Double])
-      }
-      env.fromCollection(edgeData).map(new Tuple3ToEdgeMap[Long, Double]())
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
deleted file mode 100644
index 159a100..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/package.scala
+++ /dev/null
@@ -1,30 +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
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.graph.{Graph => JGraph}
-
-import _root_.scala.reflect.ClassTag
-
-
-package object scala {
-  private[flink] def wrapGraph[K: TypeInformation : ClassTag, VV: TypeInformation : ClassTag,
-  EV: TypeInformation : ClassTag](javagraph: JGraph[K, VV, EV]) = new Graph[K, VV, EV](javagraph)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
deleted file mode 100644
index 909dbb4..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/EdgeToTuple3Map.scala
+++ /dev/null
@@ -1,30 +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.scala.utils
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.Edge
-
-@SerialVersionUID(1L)
-class EdgeToTuple3Map[K, EV] extends MapFunction[Edge[K, EV], (K, K, EV)] {
-
-  override def map(value: Edge[K, EV]): (K, K, EV) = {
-    (value.getSource, value.getTarget, value.getValue)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
deleted file mode 100644
index fd6b8c5..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple2ToVertexMap.scala
+++ /dev/null
@@ -1,30 +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.scala.utils
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.Vertex
-
-@SerialVersionUID(1L)
-class Tuple2ToVertexMap[K, VV] extends MapFunction[(K, VV), Vertex[K, VV]] {
-
-  override def map(value: (K, VV)): Vertex[K, VV] = {
-    new Vertex(value._1, value._2)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
deleted file mode 100644
index d0e07cc..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/Tuple3ToEdgeMap.scala
+++ /dev/null
@@ -1,30 +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.scala.utils
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.Edge
-
-@SerialVersionUID(1L)
-class Tuple3ToEdgeMap[K, EV] extends MapFunction[(K, K, EV), Edge[K, EV]] {
-
-  override def map(value: (K, K, EV)): Edge[K, EV] = {
-    new Edge(value._1, value._2, value._3)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala b/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
deleted file mode 100644
index faf4e10..0000000
--- a/flink-staging/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/utils/VertexToTuple2Map.scala
+++ /dev/null
@@ -1,30 +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.scala.utils
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.graph.Vertex
-
-@SerialVersionUID(1L)
-class VertexToTuple2Map[K, VV] extends MapFunction[Vertex[K, VV], (K, VV)] {
-
-  override def map(value: Vertex[K, VV]): (K, VV) = {
-    (value.getId, value.getValue)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
deleted file mode 100644
index 55faee3..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/GellyScalaAPICompletenessTest.scala
+++ /dev/null
@@ -1,43 +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.streaming.api.scala
-
-import java.lang.reflect.Method
-import org.apache.flink.graph.scala._
-import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
-import org.apache.flink.graph.{Graph => JavaGraph}
-import scala.language.existentials
-import org.junit.Test
-
-/**
- * This checks whether the Gelly Scala API is up to feature parity with the Java API.
- * Implements the {@link ScalaAPICompletenessTest} for Gelly.
- */
-class GellyScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
-
-  override def isExcludedByName(method: Method): Boolean = {
-    val name = method.getDeclaringClass.getName + "." + method.getName
-    val excludedNames = Seq("org.apache.flink.graph.Graph.getContext")
-    excludedNames.contains(name)
-  }
-
-  @Test
-  override def testCompleteness(): Unit = {
-    checkMethods("Graph", "Graph", classOf[JavaGraph[_, _, _]], classOf[Graph[_, _, _]])
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
deleted file mode 100644
index 1c2cf54..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/TestGraphUtils.scala
+++ /dev/null
@@ -1,55 +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.scala.test
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.{Edge, Vertex}
-
-object TestGraphUtils {
-
-    def getLongLongVertexData(env: ExecutionEnvironment): DataSet[Vertex[Long, Long]] = {
-        return env.fromCollection(getLongLongVertices)
-    }
-
-    def getLongLongEdgeData(env: ExecutionEnvironment): DataSet[Edge[Long, Long]] = {
-        return env.fromCollection(getLongLongEdges)
-    }
-
-    def getLongLongVertices: List[Vertex[Long, Long]] = {
-        List(
-            new Vertex[Long, Long](1L, 1L),
-            new Vertex[Long, Long](2L, 2L),
-            new Vertex[Long, Long](3L, 3L),
-            new Vertex[Long, Long](4L, 4L),
-            new Vertex[Long, Long](5L, 5L)
-        )
-    }
-
-    def getLongLongEdges: List[Edge[Long, Long]] = {
-        List(
-            new Edge[Long, Long](1L, 2L, 12L),
-            new Edge[Long, Long](1L, 3L, 13L),
-            new Edge[Long, Long](2L, 3L, 23L),
-            new Edge[Long, Long](3L, 4L, 34L),
-            new Edge[Long, Long](3L, 5L, 35L),
-            new Edge[Long, Long](4L, 5L, 45L),
-            new Edge[Long, Long](5L, 1L, 51L)
-        )
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
deleted file mode 100644
index b347049..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/DegreesITCase.scala
+++ /dev/null
@@ -1,69 +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.scala.test.operations
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class DegreesITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testInDegrees {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.inDegrees.collect().toList
-    expectedResult = "(1,1)\n" + "(2,1)\n" + "(3,2)\n" + "(4,1)\n" + "(5,2)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testOutDegrees {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.outDegrees.collect().toList
-    expectedResult = "(1,2)\n" + "(2,1)\n" + "(3,2)\n" + "(4,1)\n" + "(5,1)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testGetDegrees {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.getDegrees.collect().toList
-    expectedResult = "(1,3)\n" + "(2,2)\n" + "(3,4)\n" + "(4,2)\n" + "(5,3)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
deleted file mode 100644
index 6ceaf16..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphCreationWithCsvITCase.scala
+++ /dev/null
@@ -1,225 +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.scala.test.operations
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-import java.io.IOException
-import org.apache.flink.core.fs.FileInputSplit
-import java.io.File
-import java.io.OutputStreamWriter
-import java.io.FileOutputStream
-import java.io.FileOutputStream
-import com.google.common.base.Charsets
-import org.apache.flink.core.fs.Path
-import org.apache.flink.types.NullValue
-import org.apache.flink.api.common.functions.MapFunction
-
-@RunWith(classOf[Parameterized])
-class GraphCreationWithCsvITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvWithValues {
-    /*
-     * Test with two Csv files, both vertices and edges have values
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val verticesContent =  "1,1\n2,2\n3,3\n"
-    val verticesSplit = createTempFile(verticesContent)
-    val edgesContent =  "1,2,ot\n3,2,tt\n3,1,to\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, Long, String](
-        readVertices = true,
-        pathVertices = verticesSplit.getPath.toString,
-        pathEdges = edgesSplit.getPath.toString,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvNoEdgeValues {
-    /*
-     * Test with two Csv files; edges have no values
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val verticesContent =  "1,one\n2,two\n3,three\n"
-    val verticesSplit = createTempFile(verticesContent)
-    val edgesContent =  "1,2\n3,2\n3,1\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, String, NullValue](
-        readVertices = true,
-        pathVertices = verticesSplit.getPath.toString,
-        pathEdges = edgesSplit.getPath.toString,
-        hasEdgeValues = false,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,one,two,(null)\n3,2,three,two,(null)\n3,1,three,one,(null)\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvWithMapperValues {
-    /*
-     * Test with edges Csv file and vertex mapper initializer
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edgesContent =  "1,2,12\n3,2,32\n3,1,31\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, Double, Long](
-        readVertices = false,
-        pathEdges = edgesSplit.getPath.toString,
-        mapper = new VertexDoubleIdAssigner(),
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,1.0,2.0,12\n3,2,3.0,2.0,32\n3,1,3.0,1.0,31\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvNoVertexValues {
-    /*
-     * Test with edges Csv file: no vertex values
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edgesContent =  "1,2,12\n3,2,32\n3,1,31\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, NullValue, Long](
-        readVertices = false,
-        pathEdges = edgesSplit.getPath.toString,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,(null),(null),12\n3,2,(null),(null),32\n" +
-      "3,1,(null),(null),31\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvNoValues {
-    /*
-     * Test with edges Csv file: neither vertex nor edge values
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val edgesContent =  "1,2\n3,2\n3,1\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, NullValue, NullValue](
-        readVertices = false,
-        pathEdges = edgesSplit.getPath.toString,
-        hasEdgeValues = false,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,(null),(null),(null)\n" +
-      "3,2,(null),(null),(null)\n3,1,(null),(null),(null)\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvOptionsVertices {
-    /*
-     * Test the options for vertices: delimiters, comments, ignore first line.
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val verticesContent =  "42#42\t" + "%this-is-a-comment\t" +
-      "1#1\t" + "2#2\t" + "3#3\t"
-    val verticesSplit = createTempFile(verticesContent)
-    val edgesContent =  "1,2,ot\n3,2,tt\n3,1,to\n"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, Long, String](
-        readVertices = true,
-        pathVertices = verticesSplit.getPath.toString,
-        lineDelimiterVertices = "\t",
-        fieldDelimiterVertices = "#",
-        ignoreFirstLineVertices = true,
-        ignoreCommentsVertices = "%",
-        pathEdges = edgesSplit.getPath.toString,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testCsvOptionsEdges {
-    /*
-     * Test the options for edges: delimiters, comments, ignore first line.
-     */
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val verticesContent =  "1,1\n2,2\n3,3\n"
-    val verticesSplit = createTempFile(verticesContent)
-    val edgesContent =  "42#42#ignore&" + "1#2#ot&" + "3#2#tt&" + "3#1#to&" +
-      "//this-is-a-comment"
-    val edgesSplit = createTempFile(edgesContent)
-    val graph = Graph.fromCsvReader[Long, Long, String](
-        pathVertices = verticesSplit.getPath.toString,
-        readVertices = true,
-        lineDelimiterEdges = "&",
-        fieldDelimiterEdges = "#",
-        ignoreFirstLineEdges = true,
-        ignoreCommentsEdges = "//",
-        pathEdges = edgesSplit.getPath.toString,
-        env = env)
-    
-    val result = graph.getTriplets.collect()
-    expectedResult = "1,2,1,2,ot\n3,2,3,2,tt\n3,1,3,1,to\n"
-    TestBaseUtils.compareResultAsTuples(result.asJava, expectedResult);
-  }
-
-  @throws(classOf[IOException])
-  def createTempFile(content: String): FileInputSplit = {
-    val tempFile = File.createTempFile("test_contents", "tmp")
-    tempFile.deleteOnExit()
-
-    val wrt = new OutputStreamWriter(new FileOutputStream(tempFile), Charsets.UTF_8)
-    wrt.write(content)
-    wrt.close()
-
-    new FileInputSplit(0, new Path(tempFile.toURI.toString), 0, tempFile.length,
-        Array("localhost"));
-    }
-
-    final class VertexDoubleIdAssigner extends MapFunction[Long, Double] {
-      @throws(classOf[Exception])
-      def map(id: Long): Double = {id.toDouble}
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
deleted file mode 100644
index 4b776e2..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphMutationsITCase.scala
+++ /dev/null
@@ -1,260 +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.scala.test.operations
-
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.{Edge, Vertex}
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class GraphMutationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddVertex {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-
-    val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L))
-    val res = newgraph.getVertices.collect().toList
-    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddVertexExisting {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addVertex(new Vertex[Long, Long](1L, 1L))
-    val res = newgraph.getVertices.collect().toList
-    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddVertexNoEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addVertex(new Vertex[Long, Long](6L, 6L))
-    val res = newgraph.getVertices.collect().toList
-    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-
-    val newgraph = graph.addVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](6L, 6L),
-        new Vertex[Long, Long](7L, 7L)))
-    val res = newgraph.getVertices.collect().toList
-    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n" + "7,7\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddVerticesExisting {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-
-    val newgraph = graph.addVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](5L, 5L),
-        new Vertex[Long, Long](6L, 6L)))
-    val res = newgraph.getVertices.collect().toList
-    expectedResult = "1,1\n" + "2,2\n" + "3,3\n" + "4,4\n" + "5,5\n" + "6,6\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveVertex {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeVertex(new Vertex[Long, Long](5L, 5L))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveInvalidVertex {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeVertex(new Vertex[Long, Long](6L, 6L))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
-      "45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](1L, 1L),
-        new Vertex[Long, Long](2L, 2L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveValidAndInvalidVertex {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeVertices(List[Vertex[Long, Long]](new Vertex[Long, Long](1L, 1L),
-        new Vertex[Long, Long](6L, 6L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddEdge {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addEdge(new Vertex[Long, Long](6L, 6L), new Vertex[Long, Long](1L,
-      1L), 61L)
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
-      "45\n" + "5,1,51\n" + "6,1,61\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addEdges(List[Edge[Long, Long]](new Edge(2L, 4L, 24L),
-       new Edge(4L, 1L, 41L), new Edge(4L, 3L, 43L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "2,4,24\n" + "3,4,34\n" + "3,5," +
-    "35\n" + "4,1,41\n" + "4,3,43\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddEdgesInvalidVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addEdges(List[Edge[Long, Long]](new Edge(6L, 1L, 61L),
-       new Edge(7L, 8L, 78L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5," +
-    "35\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testAddExistingEdge {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.addEdge(new Vertex[Long, Long](1L, 1L), new Vertex[Long, Long](2L,
-      2L), 12L)
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5," +
-      "35\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveEdge {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeEdge(new Edge[Long, Long](5L, 1L, 51L))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveInvalidEdge {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeEdge(new Edge[Long, Long](6L, 1L, 61L))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
-      "45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeEdges(List[Edge[Long, Long]](new Edge(1L, 2L, 12L),
-      new Edge(4L, 5L, 45L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testRemoveSameEdgeTwiceEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val newgraph = graph.removeEdges(List[Edge[Long, Long]](new Edge(1L, 2L, 12L),
-       new Edge(1L, 2L, 12L)))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/91ffbc1e/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala b/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
deleted file mode 100644
index 7f7ebc0..0000000
--- a/flink-staging/flink-gelly-scala/src/test/scala/org/apache/flink/graph/scala/test/operations/GraphOperationsITCase.scala
+++ /dev/null
@@ -1,282 +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.scala.test.operations
-
-import org.apache.flink.api.common.functions.FilterFunction
-import org.apache.flink.api.scala._
-import org.apache.flink.graph.scala._
-import org.apache.flink.graph.scala.test.TestGraphUtils
-import org.apache.flink.graph.{Edge, Vertex}
-import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils}
-import org.junit.rules.TemporaryFolder
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-import org.junit.{After, Before, Rule, Test}
-import _root_.scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class GraphOperationsITCase(mode: MultipleProgramsTestBase.TestExecutionMode) extends
-MultipleProgramsTestBase(mode) {
-
-  private var expectedResult: String = null
-    
-  @Test
-  @throws(classOf[Exception])
-  def testUndirected {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.getUndirected.getEdges.collect().toList;
-
-    expectedResult = "1,2,12\n" + "2,1,12\n" + "1,3,13\n" + "3,1,13\n" + "2,3,23\n" + "3,2," +
-      "23\n" + "3,4,34\n" + "4,3,34\n" + "3,5,35\n" + "5,3,35\n" + "4,5,45\n" + "5,4,45\n" +
-      "5,1,51\n" + "1,5,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testReverse {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.reverse().getEdges.collect().toList;
-
-    expectedResult = "2,1,12\n" + "3,1,13\n" + "3,2,23\n" + "4,3,34\n" + "5,3,35\n" + "5,4," +
-      "45\n" + "1,5,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testSubGraph {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.subgraph(new FilterFunction[Vertex[Long, Long]] {
-      @throws(classOf[Exception])
-      def filter(vertex: Vertex[Long, Long]): Boolean = {
-        return (vertex.getValue > 2)
-      }
-    }, new FilterFunction[Edge[Long, Long]] {
-
-      @throws(classOf[Exception])
-      override def filter(edge: Edge[Long, Long]): Boolean = {
-        return (edge.getValue > 34)
-      }
-    }).getEdges.collect().toList;
-
-    expectedResult = "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testSubGraphSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.subgraph(
-      vertex => vertex.getValue > 2,
-      edge => edge.getValue > 34
-    ).getEdges.collect().toList;
-
-    expectedResult = "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testFilterOnVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.filterOnVertices(new FilterFunction[Vertex[Long, Long]] {
-      @throws(classOf[Exception])
-      def filter(vertex: Vertex[Long, Long]): Boolean = {
-        vertex.getValue > 2
-      }
-    }).getEdges.collect().toList;
-
-    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testFilterOnVerticesSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.filterOnVertices(
-      vertex => vertex.getValue > 2
-    ).getEdges.collect().toList;
-
-    expectedResult = "3,4,34\n" + "3,5,35\n" + "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testFilterOnEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.filterOnEdges(new FilterFunction[Edge[Long, Long]] {
-      @throws(classOf[Exception])
-      def filter(edge: Edge[Long, Long]): Boolean = {
-        edge.getValue > 34
-      }
-    }).getEdges.collect().toList;
-
-    expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testFilterOnEdgesSugar {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.filterOnEdges(
-      edge => edge.getValue > 34
-    ).getEdges.collect().toList;
-
-    expectedResult = "3,5,35\n" + "4,5,45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testNumberOfVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = env.fromElements(graph.numberOfVertices).collect().toList
-    expectedResult = "5"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testNumberOfEdges {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = env.fromElements(graph.numberOfEdges).collect().toList
-    expectedResult = "7"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testVertexIds {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.getVertexIds.collect().toList
-    expectedResult = "1\n2\n3\n4\n5\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testEdgesIds {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.getEdgeIds.collect().toList
-    expectedResult = "(1,2)\n" + "(1,3)\n" + "(2,3)\n" + "(3,4)\n" + "(3,5)\n" + "(4,5)\n" +
-      "(5,1)\n"
-    TestBaseUtils.compareResultAsText(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testUnion {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
-      new Vertex[Long, Long](6L, 6L)
-    )
-    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
-      new Edge[Long, Long](6L, 1L, 61L)
-    )
-
-    val newgraph = graph.union(Graph.fromCollection(vertices, edges, env))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
-      "45\n" + "5,1,51\n" + "6,1,61\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testDifference {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
-      new Vertex[Long, Long](1L, 1L), new Vertex[Long, Long](3L, 3L),
-      new Vertex[Long, Long](6L, 6L) 
-    )
-    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
-      new Edge[Long, Long](1L, 3L, 13L), new Edge[Long, Long](1L, 6L, 16L),
-      new Edge[Long, Long](6L, 3L, 63L)
-    )
-
-    val newgraph = graph.difference(Graph.fromCollection(vertices, edges, env))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "4,5,45\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testDifferenceNoCommonVertices {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val vertices: List[Vertex[Long, Long]] = List[Vertex[Long, Long]](
-      new Vertex[Long, Long](6L, 6L) 
-    )
-    val edges: List[Edge[Long, Long]] = List[Edge[Long, Long]](
-      new Edge[Long, Long](6L, 6L, 66L)
-    )
-
-    val newgraph = graph.difference(Graph.fromCollection(vertices, edges, env))
-    val res = newgraph.getEdges.collect().toList
-    expectedResult = "1,2,12\n" + "1,3,13\n" + "2,3,23\n" + "3,4,34\n" + "3,5,35\n" + "4,5," +
-      "45\n" + "5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-
-  @Test
-  @throws(classOf[Exception])
-  def testTriplets {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val graph: Graph[Long, Long, Long] = Graph.fromDataSet(TestGraphUtils
-      .getLongLongVertexData(env), TestGraphUtils.getLongLongEdgeData(env), env)
-    val res = graph.getTriplets.collect().toList
-    expectedResult = "1,2,1,2,12\n" + "1,3,1,3,13\n" + "2,3,2,3,23\n" + "3,4,3,4,34\n" +
-      "3,5,3,5,35\n" + "4,5,4,5,45\n" + "5,1,5,1,51\n"
-    TestBaseUtils.compareResultAsTuples(res.asJava, expectedResult)
-  }
-}