You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/08/30 23:44:36 UTC

[1/5] flink git commit: [tests] Improve and combine iteration tests with aggregators (less static sharing, collect(), parallel execution)

Repository: flink
Updated Branches:
  refs/heads/master 0ba53558f -> 0858d9f12


[tests] Improve and combine iteration tests with aggregators (less static sharing, collect(), parallel execution)


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

Branch: refs/heads/master
Commit: 0858d9f12101ecf7b29e995561dd3fc4ce00eb38
Parents: a7a57eb
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Aug 30 19:14:00 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Aug 30 22:39:17 2015 +0200

----------------------------------------------------------------------
 .../AggregatorConvergenceITCase.java            | 350 +++++++++++++++++++
 ...nentsWithParametrizableAggregatorITCase.java | 239 -------------
 ...entsWithParametrizableConvergenceITCase.java | 224 ------------
 3 files changed, 350 insertions(+), 463 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0858d9f1/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java
new file mode 100644
index 0000000..941b31b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java
@@ -0,0 +1,350 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.iterative.aggregators;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
+import org.apache.flink.api.common.aggregators.LongSumAggregator;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichJoinFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.util.Collector;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.junit.Assert.*;
+
+/**
+ * Connected Components test case that uses a parameterizable convergence criterion
+ */
+@RunWith(Parameterized.class)
+@SuppressWarnings("serial")
+public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
+
+	public AggregatorConvergenceITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+	
+	@Test
+	public void testConnectedComponentsWithParametrizableConvergence() {
+		try {
+			List<Tuple2<Long, Long>> verticesInput = Arrays.asList(
+					new Tuple2<Long, Long>(1l,1l),
+					new Tuple2<Long, Long>(2l,2l),
+					new Tuple2<Long, Long>(3l,3l),
+					new Tuple2<Long, Long>(4l,4l),
+					new Tuple2<Long, Long>(5l,5l),
+					new Tuple2<Long, Long>(6l,6l),
+					new Tuple2<Long, Long>(7l,7l),
+					new Tuple2<Long, Long>(8l,8l),
+					new Tuple2<Long, Long>(9l,9l)
+			);
+			
+			List<Tuple2<Long, Long>> edgesInput = Arrays.asList(
+					new Tuple2<Long, Long>(1l,2l),
+					new Tuple2<Long, Long>(1l,3l),
+					new Tuple2<Long, Long>(2l,3l),
+					new Tuple2<Long, Long>(2l,4l),
+					new Tuple2<Long, Long>(2l,1l),
+					new Tuple2<Long, Long>(3l,1l),
+					new Tuple2<Long, Long>(3l,2l),
+					new Tuple2<Long, Long>(4l,2l),
+					new Tuple2<Long, Long>(4l,6l),
+					new Tuple2<Long, Long>(5l,6l),
+					new Tuple2<Long, Long>(6l,4l),
+					new Tuple2<Long, Long>(6l,5l),
+					new Tuple2<Long, Long>(7l,8l),
+					new Tuple2<Long, Long>(7l,9l),
+					new Tuple2<Long, Long>(8l,7l),
+					new Tuple2<Long, Long>(8l,9l),
+					new Tuple2<Long, Long>(9l,7l),
+					new Tuple2<Long, Long>(9l,8l)
+			);
+
+			// name of the aggregator that checks for convergence
+			final String UPDATED_ELEMENTS = "updated.elements.aggr";
+
+			// the iteration stops if less than this number os elements change value
+			final long convergence_threshold = 3;
+
+			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
+			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
+
+			IterativeDataSet<Tuple2<Long, Long>> iteration =
+					initialSolutionSet.iterate(10);
+
+			// register the convergence criterion
+			iteration.registerAggregationConvergenceCriterion(UPDATED_ELEMENTS,
+					new LongSumAggregator(), new UpdatedElementsConvergenceCriterion(convergence_threshold));
+
+			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.join(edges).where(0).equalTo(0)
+					.with(new NeighborWithComponentIDJoin())
+					.groupBy(0).min(1);
+
+			DataSet<Tuple2<Long, Long>> updatedComponentId =
+					verticesWithNewComponents.join(iteration).where(0).equalTo(0)
+							.flatMap(new MinimumIdFilter(UPDATED_ELEMENTS));
+
+			List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId).collect();
+			Collections.sort(result, new JavaProgramTestBase.TupleComparator<Tuple2<Long, Long>>());
+
+			List<Tuple2<Long, Long>> expectedResult = Arrays.asList(
+					new Tuple2<Long, Long>(1L,1L),
+					new Tuple2<Long, Long>(2L,1L),
+					new Tuple2<Long, Long>(3L,1L),
+					new Tuple2<Long, Long>(4L,1L),
+					new Tuple2<Long, Long>(5L,2L),
+					new Tuple2<Long, Long>(6L,1L),
+					new Tuple2<Long, Long>(7L,7L),
+					new Tuple2<Long, Long>(8L,7L),
+					new Tuple2<Long, Long>(9L,7L)
+			);
+			
+			assertEquals(expectedResult, result);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testParameterizableAggregator() {
+		try {
+			List<Tuple2<Long, Long>> verticesInput = Arrays.asList(
+				new Tuple2<Long, Long>(1l,1l),
+				new Tuple2<Long, Long>(2l,2l),
+				new Tuple2<Long, Long>(3l,3l),
+				new Tuple2<Long, Long>(4l,4l),
+				new Tuple2<Long, Long>(5l,5l),
+				new Tuple2<Long, Long>(6l,6l),
+				new Tuple2<Long, Long>(7l,7l),
+				new Tuple2<Long, Long>(8l,8l),
+				new Tuple2<Long, Long>(9l,9l)
+			);
+			
+			List<Tuple2<Long, Long>> edgesInput = Arrays.asList(
+					new Tuple2<>(1l,2l),
+					new Tuple2<>(1l,3l),
+					new Tuple2<>(2l,3l),
+					new Tuple2<>(2l,4l),
+					new Tuple2<>(2l,1l),
+					new Tuple2<>(3l,1l),
+					new Tuple2<>(3l,2l),
+					new Tuple2<>(4l,2l),
+					new Tuple2<>(4l,6l),
+					new Tuple2<>(5l,6l),
+					new Tuple2<>(6l,4l),
+					new Tuple2<>(6l,5l),
+					new Tuple2<>(7l,8l),
+					new Tuple2<>(7l,9l),
+					new Tuple2<>(8l,7l),
+					new Tuple2<>(8l,9l),
+					new Tuple2<>(9l,7l),
+					new Tuple2<>(9l,8l)
+			);
+
+			final int MAX_ITERATIONS = 5;
+			final String AGGREGATOR_NAME = "elements.in.component.aggregator";
+			final long componentId = 1l;
+
+			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
+			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
+
+			IterativeDataSet<Tuple2<Long, Long>> iteration =
+					initialSolutionSet.iterate(MAX_ITERATIONS);
+
+			// register the aggregator
+			iteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregatorWithParameter(componentId));
+
+			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.join(edges).where(0).equalTo(0)
+					.with(new NeighborWithComponentIDJoin())
+					.groupBy(0).min(1);
+
+			DataSet<Tuple2<Long, Long>> updatedComponentId =
+					verticesWithNewComponents.join(iteration).where(0).equalTo(0)
+							.flatMap(new MinimumIdFilterCounting(AGGREGATOR_NAME));
+
+			List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId).collect();
+
+			Collections.sort(result, new JavaProgramTestBase.TupleComparator<Tuple2<Long, Long>>());
+
+			List<Tuple2<Long, Long>> expectedResult = Arrays.asList(
+					new Tuple2<>(1L,1L),
+					new Tuple2<>(2L,1L),
+					new Tuple2<>(3L,1L),
+					new Tuple2<>(4L,1L),
+					new Tuple2<>(5L,1L),
+					new Tuple2<>(6L,1L),
+					new Tuple2<>(7L,7L),
+					new Tuple2<>(8L,7L),
+					new Tuple2<>(9L,7L)
+			);
+
+			// checkpogram result
+			assertEquals(expectedResult, result);
+
+			// check aggregators
+			long[] aggr_values = MinimumIdFilterCounting.aggr_value;
+
+			// note that position 0 has the end result from superstep 1, retrieved at the start of iteration 2
+			// position one as superstep 2, retrieved at the start of iteration 3.
+			// the result from iteration 5 is not available, because no iteration 6 happens
+			assertEquals(3, aggr_values[0]);
+			assertEquals(4, aggr_values[1]);
+			assertEquals(5, aggr_values[2]);
+			assertEquals(6, aggr_values[3]);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Test Functions
+	// ------------------------------------------------------------------------
+
+	public static final class NeighborWithComponentIDJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithCompId, Tuple2<Long, Long> edge) {
+			vertexWithCompId.f0 = edge.f1;
+			return vertexWithCompId;
+		}
+	}
+	
+	public static class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
+
+		private final String aggName;
+		private LongSumAggregator aggr;
+
+		public MinimumIdFilter(String aggName) {
+			this.aggName = aggName;
+		}
+
+		@Override
+		public void open(Configuration conf) {
+			aggr = getIterationRuntimeContext().getIterationAggregator(aggName);
+		}
+
+		@Override
+		public void flatMap(
+				Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> vertexWithNewAndOldId,
+				Collector<Tuple2<Long, Long>> out) {
+
+			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
+				out.collect(vertexWithNewAndOldId.f0);
+				aggr.aggregate(1l);
+			}
+			else {
+				out.collect(vertexWithNewAndOldId.f1);
+			}
+		}
+	}
+
+	public static final class MinimumIdFilterCounting 
+			extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
+		
+		private static final long[] aggr_value = new long[5];
+		
+		private final String aggName;
+		private LongSumAggregatorWithParameter aggr;
+
+		public MinimumIdFilterCounting(String aggName) {
+			this.aggName = aggName;
+		}
+		
+		@Override
+		public void open(Configuration conf) {
+			final int superstep = getIterationRuntimeContext().getSuperstepNumber();
+			
+			aggr = getIterationRuntimeContext().getIterationAggregator(aggName);
+			
+			if (superstep > 1 && getIterationRuntimeContext().getIndexOfThisSubtask() == 0) {
+				LongValue val = getIterationRuntimeContext().getPreviousIterationAggregate(aggName);
+				aggr_value[superstep - 2] = val.getValue();
+			}
+		}
+
+		@Override
+		public void flatMap(
+				Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> vertexWithNewAndOldId,
+				Collector<Tuple2<Long, Long>> out) {
+
+			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
+				out.collect(vertexWithNewAndOldId.f0);
+				if (vertexWithNewAndOldId.f0.f1 == aggr.getComponentId()) {
+					aggr.aggregate(1l);
+				}
+			} else {
+				out.collect(vertexWithNewAndOldId.f1);
+				if (vertexWithNewAndOldId.f1.f1 == aggr.getComponentId()) {
+					aggr.aggregate(1l);
+				}
+			}
+		}
+	}
+
+	/** A Convergence Criterion with one parameter */
+	public static class UpdatedElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
+
+		private final long threshold;
+
+		public UpdatedElementsConvergenceCriterion(long u_threshold) {
+			this.threshold = u_threshold;
+		}
+
+		@Override
+		public boolean isConverged(int iteration, LongValue value) {
+			return value.getValue() < this.threshold;
+		}
+	}
+
+	public static final class LongSumAggregatorWithParameter extends LongSumAggregator {
+
+		private long componentId;
+
+		public LongSumAggregatorWithParameter(long compId) {
+			this.componentId = compId;
+		}
+
+		public long getComponentId() {
+			return this.componentId;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0858d9f1/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
deleted file mode 100644
index 8bf50de..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableAggregatorITCase.java
+++ /dev/null
@@ -1,239 +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.test.iterative.aggregators;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-
-/**
- * Connected Components test case that uses a parameterizable aggregator
- */
-public class ConnectedComponentsWithParametrizableAggregatorITCase extends JavaProgramTestBase {
-
-	private static final int MAX_ITERATIONS = 5;
-	private static final int parallelism = 1;
-
-	protected static List<Tuple2<Long, Long>> verticesInput = new ArrayList<Tuple2<Long, Long>>();
-	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
-	private String resultPath;
-	private String expectedResult;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		// vertices input
-		verticesInput.clear();
-		verticesInput.add(new Tuple2<Long, Long>(1l,1l));
-		verticesInput.add(new Tuple2<Long, Long>(2l,2l));
-		verticesInput.add(new Tuple2<Long, Long>(3l,3l));
-		verticesInput.add(new Tuple2<Long, Long>(4l,4l));
-		verticesInput.add(new Tuple2<Long, Long>(5l,5l));
-		verticesInput.add(new Tuple2<Long, Long>(6l,6l));
-		verticesInput.add(new Tuple2<Long, Long>(7l,7l));
-		verticesInput.add(new Tuple2<Long, Long>(8l,8l));
-		verticesInput.add(new Tuple2<Long, Long>(9l,9l));
-
-		// vertices input
-		edgesInput.clear();
-		edgesInput.add(new Tuple2<Long, Long>(1l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(1l,3l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,3l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,4l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,1l));
-		edgesInput.add(new Tuple2<Long, Long>(3l,1l));
-		edgesInput.add(new Tuple2<Long, Long>(3l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(4l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(4l,6l));
-		edgesInput.add(new Tuple2<Long, Long>(5l,6l));
-		edgesInput.add(new Tuple2<Long, Long>(6l,4l));
-		edgesInput.add(new Tuple2<Long, Long>(6l,5l));
-		edgesInput.add(new Tuple2<Long, Long>(7l,8l));
-		edgesInput.add(new Tuple2<Long, Long>(7l,9l));
-		edgesInput.add(new Tuple2<Long, Long>(8l,7l));
-		edgesInput.add(new Tuple2<Long, Long>(8l,9l));
-		edgesInput.add(new Tuple2<Long, Long>(9l,7l));
-		edgesInput.add(new Tuple2<Long, Long>(9l,8l));
-
-		resultPath = getTempDirPath("result");
-
-		expectedResult = "(1,1)\n" + "(2,1)\n" + "(3,1)\n" + "(4,1)\n" +
-						"(5,1)\n" + "(6,1)\n" + "(7,7)\n" + "(8,7)\n" + "(9,7)\n";
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ConnectedComponentsWithAggregatorProgram.runProgram(resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(expectedResult, resultPath);
-		long[] aggr_values = ConnectedComponentsWithAggregatorProgram.aggr_value;
-		
-		// note that position 0 has the end result from superstep 1, retrieved at the start of iteration 2
-		// position one as superstep 2, retrieved at the start of iteration 3.
-		// the result from iteration 5 is not available, because no iteration 6 happens
-		Assert.assertEquals(3, aggr_values[0]);
-		Assert.assertEquals(4, aggr_values[1]);
-		Assert.assertEquals(5, aggr_values[2]);
-		Assert.assertEquals(6, aggr_values[3]);
-	}
-
-
-	private static class ConnectedComponentsWithAggregatorProgram {
-
-		private static final String ELEMENTS_IN_COMPONENT = "elements.in.component.aggregator";
-		private static final long componentId = 1l;
-		private static long [] aggr_value = new long [MAX_ITERATIONS];
-
-		public static String runProgram(String resultPath) throws Exception {
-
-			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(parallelism);
-
-			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
-			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
-
-			IterativeDataSet<Tuple2<Long, Long>> iteration =
-					initialSolutionSet.iterate(MAX_ITERATIONS);
-
-			// register the aggregator
-			iteration.registerAggregator(ELEMENTS_IN_COMPONENT, new LongSumAggregatorWithParameter(componentId));
-
-			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.join(edges).where(0).equalTo(0)
-					.with(new NeighborWithComponentIDJoin())
-					.groupBy(0).reduceGroup(new MinimumReduce());
-
-			DataSet<Tuple2<Long, Long>> updatedComponentId =
-					verticesWithNewComponents.join(iteration).where(0).equalTo(0)
-					.flatMap(new MinimumIdFilter());
-
-			iteration.closeWith(updatedComponentId).writeAsText(resultPath);
-
-			env.execute();
-
-			return resultPath;
-		}
-	}
-
-	public static final class NeighborWithComponentIDJoin implements JoinFunction
-		<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithCompId,
-				Tuple2<Long, Long> edge) throws Exception {
-
-			vertexWithCompId.setField(edge.f1, 0);
-			return vertexWithCompId;
-		}
-	}
-
-	public static final class MinimumReduce extends RichGroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-		
-		private final Tuple2<Long, Long> resultVertex = new Tuple2<Long, Long>();
-
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
-			Long vertexId = 0L;
-			Long minimumCompId = Long.MAX_VALUE;
-
-			for (Tuple2<Long, Long> value: values) {
-				vertexId = value.f0;
-				Long candidateCompId = value.f1;
-				if (candidateCompId < minimumCompId) {
-					minimumCompId = candidateCompId;
-				}
-			}
-			resultVertex.f0 = vertexId;
-			resultVertex.f1 = minimumCompId;
-
-			out.collect(resultVertex);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
-
-		private static LongSumAggregatorWithParameter aggr;
-
-		@Override
-		public void open(Configuration conf) {
-			aggr = getIterationRuntimeContext().getIterationAggregator(
-					ConnectedComponentsWithAggregatorProgram.ELEMENTS_IN_COMPONENT);
-
-			int superstep = getIterationRuntimeContext().getSuperstepNumber(); 
-
-			if (superstep > 1) {
-				LongValue val = getIterationRuntimeContext().getPreviousIterationAggregate(
-						ConnectedComponentsWithAggregatorProgram.ELEMENTS_IN_COMPONENT);
-				ConnectedComponentsWithAggregatorProgram.aggr_value[superstep-2] = val.getValue();
-			}
-		}
-
-		@Override
-		public void flatMap(
-				Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> vertexWithNewAndOldId,
-				Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
-				out.collect(vertexWithNewAndOldId.f0);
-				if (vertexWithNewAndOldId.f0.f1 == aggr.getComponentId()) {
-					aggr.aggregate(1l);
-				}
-			} else {
-				out.collect(vertexWithNewAndOldId.f1);
-				if (vertexWithNewAndOldId.f1.f1 == aggr.getComponentId()) {
-					aggr.aggregate(1l);
-				}
-			}
-		}
-	}
-
-	// A LongSumAggregator with one parameter
-	@SuppressWarnings("serial")
-	public static final class LongSumAggregatorWithParameter extends LongSumAggregator {
-
-		private long componentId;
-
-		public LongSumAggregatorWithParameter(long compId) {
-			this.componentId = compId;
-		}
-
-		public long getComponentId() {
-			return this.componentId;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0858d9f1/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
deleted file mode 100644
index e616a2b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/ConnectedComponentsWithParametrizableConvergenceITCase.java
+++ /dev/null
@@ -1,224 +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.test.iterative.aggregators;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.common.functions.RichJoinFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-
-
-/**
- * 
- * Connected Components test case that uses a parametrizable convergence criterion
- *
- */
-public class ConnectedComponentsWithParametrizableConvergenceITCase extends JavaProgramTestBase {
-
-	private static final int MAX_ITERATIONS = 10;
-	private static final int parallelism = 1;
-
-	protected static List<Tuple2<Long, Long>> verticesInput = new ArrayList<Tuple2<Long, Long>>();
-	protected static List<Tuple2<Long, Long>> edgesInput = new ArrayList<Tuple2<Long, Long>>();
-	private String resultPath;
-	private String expectedResult;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		// vertices input
-		verticesInput.clear();
-		verticesInput.add(new Tuple2<Long, Long>(1l,1l));
-		verticesInput.add(new Tuple2<Long, Long>(2l,2l));
-		verticesInput.add(new Tuple2<Long, Long>(3l,3l));
-		verticesInput.add(new Tuple2<Long, Long>(4l,4l));
-		verticesInput.add(new Tuple2<Long, Long>(5l,5l));
-		verticesInput.add(new Tuple2<Long, Long>(6l,6l));
-		verticesInput.add(new Tuple2<Long, Long>(7l,7l));
-		verticesInput.add(new Tuple2<Long, Long>(8l,8l));
-		verticesInput.add(new Tuple2<Long, Long>(9l,9l));
-
-		// vertices input
-		edgesInput.clear();
-		edgesInput.add(new Tuple2<Long, Long>(1l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(1l,3l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,3l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,4l));
-		edgesInput.add(new Tuple2<Long, Long>(2l,1l));
-		edgesInput.add(new Tuple2<Long, Long>(3l,1l));
-		edgesInput.add(new Tuple2<Long, Long>(3l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(4l,2l));
-		edgesInput.add(new Tuple2<Long, Long>(4l,6l));
-		edgesInput.add(new Tuple2<Long, Long>(5l,6l));
-		edgesInput.add(new Tuple2<Long, Long>(6l,4l));
-		edgesInput.add(new Tuple2<Long, Long>(6l,5l));
-		edgesInput.add(new Tuple2<Long, Long>(7l,8l));
-		edgesInput.add(new Tuple2<Long, Long>(7l,9l));
-		edgesInput.add(new Tuple2<Long, Long>(8l,7l));
-		edgesInput.add(new Tuple2<Long, Long>(8l,9l));
-		edgesInput.add(new Tuple2<Long, Long>(9l,7l));
-		edgesInput.add(new Tuple2<Long, Long>(9l,8l));
-
-		resultPath = getTempDirPath("result");
-
-		expectedResult = "(1,1)\n" + "(2,1)\n" + "(3,1)\n" + "(4,1)\n" +
-						"(5,2)\n" + "(6,1)\n" + "(7,7)\n" + "(8,7)\n" + "(9,7)\n";
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ConnectedComponentsWithConvergenceProgram.runProgram(resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(expectedResult, resultPath);
-	}
-
-
-	private static class ConnectedComponentsWithConvergenceProgram {
-
-		private static final String UPDATED_ELEMENTS = "updated.elements.aggr";
-		private static final long convergence_threshold = 3; // the iteration stops if less than this number os elements change value
-
-		public static String runProgram(String resultPath) throws Exception {
-
-			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(parallelism);
-
-			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
-			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
-
-			IterativeDataSet<Tuple2<Long, Long>> iteration =
-					initialSolutionSet.iterate(MAX_ITERATIONS);
-
-			// register the convergence criterion
-			iteration.registerAggregationConvergenceCriterion(UPDATED_ELEMENTS,
-					new LongSumAggregator(), new UpdatedElementsConvergenceCriterion(convergence_threshold));
-
-			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.join(edges).where(0).equalTo(0)
-					.with(new NeighborWithComponentIDJoin())
-					.groupBy(0).reduceGroup(new MinimumReduce());
-
-			DataSet<Tuple2<Long, Long>> updatedComponentId = 
-					verticesWithNewComponents.join(iteration).where(0).equalTo(0)
-					.flatMap(new MinimumIdFilter());
-
-			iteration.closeWith(updatedComponentId).writeAsText(resultPath);
-
-			env.execute();
-
-			return resultPath;
-		}
-	}
-
-	public static final class NeighborWithComponentIDJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Long, Long> join(Tuple2<Long, Long> vertexWithCompId,
-				Tuple2<Long, Long> edge) throws Exception {
-
-			vertexWithCompId.setField(edge.f1, 0);
-			return vertexWithCompId;
-		}
-	}
-
-	public static final class MinimumReduce extends RichGroupReduceFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-		final Tuple2<Long, Long> resultVertex = new Tuple2<Long, Long>();
-
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
-			Long vertexId = 0L;
-			Long minimumCompId = Long.MAX_VALUE;
-
-			for (Tuple2<Long, Long> value: values) {
-				vertexId = value.f0;
-				Long candidateCompId = value.f1;
-				if (candidateCompId < minimumCompId) {
-					minimumCompId = candidateCompId;
-				}
-			}
-			resultVertex.f0 = vertexId;
-			resultVertex.f1 = minimumCompId;
-
-			out.collect(resultVertex);
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static final class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
-
-		private static LongSumAggregator aggr;
-
-		@Override
-		public void open(Configuration conf) {
-			aggr = getIterationRuntimeContext().getIterationAggregator(
-					ConnectedComponentsWithConvergenceProgram.UPDATED_ELEMENTS);
-		}
-
-		@Override
-		public void flatMap(
-				Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> vertexWithNewAndOldId,
-				Collector<Tuple2<Long, Long>> out) throws Exception {
-
-			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
-				out.collect(vertexWithNewAndOldId.f0);
-				aggr.aggregate(1l);
-			} else {
-				out.collect(vertexWithNewAndOldId.f1);
-			}
-		}
-	}
-
-	// A Convergence Criterion with one parameter
-	@SuppressWarnings("serial")
-	public static final class UpdatedElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
-
-		private long threshold;
-
-		public UpdatedElementsConvergenceCriterion(long u_threshold) {
-			this.threshold = u_threshold;
-		}
-
-		public long getThreshold() {
-			return this.threshold;
-		}
-
-		@Override
-		public boolean isConverged(int iteration, LongValue value) {
-			return value.getValue() < this.threshold;
-		}
-	}
-
-}


[2/5] flink git commit: [FLINK-1681] [tests] Remove outdated 'nephele' iteration tests.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStats.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStats.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStats.java
deleted file mode 100644
index e093a48..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStats.java
+++ /dev/null
@@ -1,124 +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.test.iterative.nephele.danglingpagerank;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-
-public class PageRankStats implements Value {
-	private static final long serialVersionUID = 1L;
-
-	private double diff;
-
-	private double rank;
-
-	private double danglingRank;
-
-	private long numDanglingVertices;
-
-	private long numVertices;
-
-	private long edges;
-
-	private double summedRank;
-
-	private double finalDiff;
-
-	public PageRankStats() {
-	}
-
-	public PageRankStats(double diff, double rank, double danglingRank, long numDanglingVertices, long numVertices,
-			long edges, double summedRank, double finalDiff) {
-		this.diff = diff;
-		this.rank = rank;
-		this.danglingRank = danglingRank;
-		this.numDanglingVertices = numDanglingVertices;
-		this.numVertices = numVertices;
-		this.edges = edges;
-		this.summedRank = summedRank;
-		this.finalDiff = finalDiff;
-	}
-
-	public double diff() {
-		return diff;
-	}
-
-	public double rank() {
-		return rank;
-	}
-
-	public double danglingRank() {
-		return danglingRank;
-	}
-
-	public long numDanglingVertices() {
-		return numDanglingVertices;
-	}
-
-	public long numVertices() {
-		return numVertices;
-	}
-
-	public long edges() {
-		return edges;
-	}
-
-	public double summedRank() {
-		return summedRank;
-	}
-
-	public double finalDiff() {
-		return finalDiff;
-	}
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		out.writeDouble(diff);
-		out.writeDouble(rank);
-		out.writeDouble(danglingRank);
-		out.writeLong(numDanglingVertices);
-		out.writeLong(numVertices);
-		out.writeLong(edges);
-		out.writeDouble(summedRank);
-		out.writeDouble(finalDiff);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		diff = in.readDouble();
-		rank = in.readDouble();
-		danglingRank = in.readDouble();
-		numDanglingVertices = in.readLong();
-		numVertices = in.readLong();
-		edges = in.readLong();
-		summedRank = in.readDouble();
-		finalDiff = in.readDouble();
-	}
-
-	@Override
-	public String toString() {
-		return "PageRankStats: diff [" + diff + "], rank [" + rank + "], danglingRank [" + danglingRank +
-			"], numDanglingVertices [" + numDanglingVertices + "], numVertices [" + numVertices + "], edges [" + edges +
-			"], summedRank [" + summedRank + "], finalDiff [" + finalDiff + "]";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStatsAggregator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStatsAggregator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStatsAggregator.java
deleted file mode 100644
index 4b41e45..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageRankStatsAggregator.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.test.iterative.nephele.danglingpagerank;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-
-@SuppressWarnings("serial")
-public class PageRankStatsAggregator implements Aggregator<PageRankStats> {
-
-	private double diff = 0;
-
-	private double rank = 0;
-
-	private double danglingRank = 0;
-
-	private long numDanglingVertices = 0;
-
-	private long numVertices = 0;
-
-	private long edges = 0;
-
-	private double summedRank = 0;
-
-	private double finalDiff = 0;
-
-	@Override
-	public PageRankStats getAggregate() {
-		return new PageRankStats(diff, rank, danglingRank, numDanglingVertices, numVertices, edges, summedRank,
-			finalDiff);
-	}
-
-	public void aggregate(double diffDelta, double rankDelta, double danglingRankDelta, long danglingVerticesDelta,
-			long verticesDelta, long edgesDelta, double summedRankDelta, double finalDiffDelta) {
-		diff += diffDelta;
-		rank += rankDelta;
-		danglingRank += danglingRankDelta;
-		numDanglingVertices += danglingVerticesDelta;
-		numVertices += verticesDelta;
-		edges += edgesDelta;
-		summedRank += summedRankDelta;
-		finalDiff += finalDiffDelta;
-	}
-
-	@Override
-	public void aggregate(PageRankStats pageRankStats) {
-		diff += pageRankStats.diff();
-		rank += pageRankStats.rank();
-		danglingRank += pageRankStats.danglingRank();
-		numDanglingVertices += pageRankStats.numDanglingVertices();
-		numVertices += pageRankStats.numVertices();
-		edges += pageRankStats.edges();
-		summedRank += pageRankStats.summedRank();
-		finalDiff += pageRankStats.finalDiff();
-	}
-
-	@Override
-	public void reset() {
-		diff = 0;
-		rank = 0;
-		danglingRank = 0;
-		numDanglingVertices = 0;
-		numVertices = 0;
-		edges = 0;
-		summedRank = 0;
-		finalDiff = 0;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageWithRankOutFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageWithRankOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageWithRankOutFormat.java
deleted file mode 100644
index 5cd520f..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/PageWithRankOutFormat.java
+++ /dev/null
@@ -1,47 +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.test.iterative.nephele.danglingpagerank;
-
-import com.google.common.base.Charsets;
-
-import java.io.IOException;
-
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class PageWithRankOutFormat extends FileOutputFormat {
-  private static final long serialVersionUID = 1L;
-
-  private final StringBuilder buffer = new StringBuilder();
-
-  @Override
-  public void writeRecord(Record record) throws IOException {
-    buffer.setLength(0);
-    buffer.append(record.getField(0, LongValue.class).toString());
-    buffer.append('\t');
-    buffer.append(record.getField(1, DoubleValue.class).toString());
-    buffer.append('\n');
-
-    byte[] bytes = buffer.toString().getBytes(Charsets.UTF_8);
-    stream.write(bytes);
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
deleted file mode 100644
index 80ba91a..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/DanglingPageRank.java
+++ /dev/null
@@ -1,105 +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.test.recordJobs.graph;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.Program;
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.java.record.operators.BulkIteration;
-import org.apache.flink.api.java.record.operators.CoGroupOperator;
-import org.apache.flink.api.java.record.operators.FileDataSink;
-import org.apache.flink.api.java.record.operators.FileDataSource;
-import org.apache.flink.api.java.record.operators.JoinOperator;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.DanglingPageRankInputFormat;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.DiffL1NormConvergenceCriterion;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.DotProductCoGroup;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.DotProductMatch;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.ImprovedAdjacencyListInputFormat;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.PageRankStatsAggregator;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.PageWithRankOutFormat;
-import org.apache.flink.types.LongValue;
-
-@SuppressWarnings("deprecation")
-public class DanglingPageRank implements Program, ProgramDescription {
-	
-	private static final long serialVersionUID = 1L;
-	
-	public static final String NUM_VERTICES_CONFIG_PARAM = "pageRank.numVertices";
-		
-	public Plan getPlan(String ... args) {
-		int parallelism = 1;
-		String pageWithRankInputPath = "";
-		String adjacencyListInputPath = "";
-		String outputPath = "";
-		int numIterations = 25;
-		long numVertices = 5;
-		long numDanglingVertices = 1;
-
-		if (args.length >= 7) {
-			parallelism = Integer.parseInt(args[0]);
-			pageWithRankInputPath = args[1];
-			adjacencyListInputPath = args[2];
-			outputPath = args[3];
-			numIterations = Integer.parseInt(args[4]);
-			numVertices = Long.parseLong(args[5]);
-			numDanglingVertices = Long.parseLong(args[6]);
-		}
-		
-		FileDataSource pageWithRankInput = new FileDataSource(new DanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput");
-		pageWithRankInput.getParameters().setLong(DanglingPageRankInputFormat.NUM_VERTICES_PARAMETER, numVertices);
-		
-		BulkIteration iteration = new BulkIteration("Page Rank Loop");
-		iteration.setInput(pageWithRankInput);
-		
-		FileDataSource adjacencyListInput = new FileDataSource(new ImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput");
-		
-		JoinOperator join = JoinOperator.builder(new DotProductMatch(), LongValue.class, 0, 0)
-				.input1(iteration.getPartialSolution())
-				.input2(adjacencyListInput)
-				.name("Join with Edges")
-				.build();
-		
-		CoGroupOperator rankAggregation = CoGroupOperator.builder(new DotProductCoGroup(), LongValue.class, 0, 0)
-				.input1(iteration.getPartialSolution())
-				.input2(join)
-				.name("Rank Aggregation")
-				.build();
-		rankAggregation.getParameters().setLong(DotProductCoGroup.NUM_VERTICES_PARAMETER, numVertices);
-		rankAggregation.getParameters().setLong(DotProductCoGroup.NUM_DANGLING_VERTICES_PARAMETER, numDanglingVertices);
-		
-		iteration.setNextPartialSolution(rankAggregation);
-		iteration.setMaximumNumberOfIterations(numIterations);
-		iteration.getAggregators().registerAggregationConvergenceCriterion(DotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator(), 
-				new DiffL1NormConvergenceCriterion());
-		
-		FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks");
-
-		Plan p = new Plan(out, "Dangling PageRank");
-		p.setDefaultParallelism(parallelism);
-		return p;
-	}
-
-	@Override
-	public String getDescription() {
-		return "Parameters: <parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
deleted file mode 100644
index 31d992f..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/SimplePageRank.java
+++ /dev/null
@@ -1,194 +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.test.recordJobs.graph;
-
-import java.io.Serializable;
-import java.util.Iterator;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.Program;
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.java.record.functions.JoinFunction;
-import org.apache.flink.api.java.record.functions.ReduceFunction;
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields;
-import org.apache.flink.api.java.record.operators.BulkIteration;
-import org.apache.flink.api.java.record.operators.FileDataSink;
-import org.apache.flink.api.java.record.operators.FileDataSource;
-import org.apache.flink.api.java.record.operators.JoinOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.DanglingPageRankInputFormat;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.ImprovedAdjacencyListInputFormat;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.LongArrayView;
-import org.apache.flink.test.recordJobs.graph.pageRankUtil.PageWithRankOutFormat;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("deprecation")
-public class SimplePageRank implements Program, ProgramDescription {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private static final String NUM_VERTICES_CONFIG_PARAM = "pageRank.numVertices";
-	
-	// --------------------------------------------------------------------------------------------
-
-	public static final class JoinVerexWithEdgesMatch extends JoinFunction implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		private Record record = new Record();
-		private LongValue vertexID = new LongValue();
-		private DoubleValue partialRank = new DoubleValue();
-		private DoubleValue rank = new DoubleValue();
-
-		private LongArrayView adjacentNeighbors = new LongArrayView();
-		
-		@Override
-		public void join(Record pageWithRank, Record edges, Collector<Record> out) throws Exception {
-			rank = pageWithRank.getField(1, rank);
-			adjacentNeighbors = edges.getField(1, adjacentNeighbors);
-			int numNeighbors = adjacentNeighbors.size();
-
-			double rankToDistribute = rank.getValue() / (double) numNeighbors;
-
-			partialRank.setValue(rankToDistribute);
-			record.setField(1, partialRank);
-			
-			for (int n = 0; n < numNeighbors; n++) {
-				vertexID.setValue(adjacentNeighbors.getQuick(n));
-				record.setField(0, vertexID);
-				out.collect(record);
-			}
-		}
-	}
-	
-	@Combinable
-	@ConstantFields(0)
-	public static final class AggregatingReduce extends ReduceFunction implements Serializable {
-		private static final long serialVersionUID = 1L;
-		
-		private final DoubleValue sum = new DoubleValue();
-
-		@Override
-		public void reduce(Iterator<Record> pageWithPartialRank, Collector<Record> out) throws Exception {
-			Record rec = null;
-			double rankSum = 0.0;
-			
-			while (pageWithPartialRank.hasNext()) {
-				rec = pageWithPartialRank.next();
-				rankSum += rec.getField(1, DoubleValue.class).getValue();
-			}
-			sum.setValue(rankSum);
-			
-			rec.setField(1, sum);
-			out.collect(rec);
-		}
-	}
-	
-	public static final class JoinOldAndNew extends JoinFunction implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		private Record record = new Record();
-		private LongValue vertexID = new LongValue();
-		private DoubleValue newRank = new DoubleValue();
-		private DoubleValue rank = new DoubleValue();
-		
-		@Override
-		public void join(Record pageWithRank, Record newPageWithRank, Collector<Record> out) throws Exception {
-			rank = pageWithRank.getField(1, rank);
-			newRank = newPageWithRank.getField(1, newRank);
-			vertexID = pageWithRank.getField(0, vertexID);
-			
-			double epsilon = 0.05;
-			double criterion = rank.getValue() - newRank.getValue();
-			
-			if(Math.abs(criterion) > epsilon)
-			{
-				record.setField(0, new IntValue(1));
-				out.collect(record);
-			}
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public Plan getPlan(String ... args) {
-		int parallelism = 1;
-		String pageWithRankInputPath = "";
-		String adjacencyListInputPath = "";
-		String outputPath = "";
-		int numIterations = 25;
-		long numVertices = 5;
-
-		if (args.length >= 6) {
-			parallelism = Integer.parseInt(args[0]);
-			pageWithRankInputPath = args[1];
-			adjacencyListInputPath = args[2];
-			outputPath = args[3];
-			numIterations = Integer.parseInt(args[4]);
-			numVertices = Long.parseLong(args[5]);
-		}
-		
-		FileDataSource pageWithRankInput = new FileDataSource(new DanglingPageRankInputFormat(),
-			pageWithRankInputPath, "PageWithRank Input");
-		pageWithRankInput.getParameters().setLong(NUM_VERTICES_CONFIG_PARAM, numVertices);
-		
-		BulkIteration iteration = new BulkIteration("Page Rank Loop");
-		iteration.setInput(pageWithRankInput);
-		
-		FileDataSource adjacencyListInput = new FileDataSource(new ImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput");
-		
-		JoinOperator join = JoinOperator.builder(new JoinVerexWithEdgesMatch(), LongValue.class, 0, 0)
-				.input1(iteration.getPartialSolution())
-				.input2(adjacencyListInput)
-				.name("Join with Edges")
-				.build();
-		
-		ReduceOperator rankAggregation = ReduceOperator.builder(new AggregatingReduce(), LongValue.class, 0)
-				.input(join)
-				.name("Rank Aggregation")
-				.build();
-		
-		iteration.setNextPartialSolution(rankAggregation);
-		iteration.setMaximumNumberOfIterations(numIterations);
-		
-		JoinOperator termination = JoinOperator.builder(new JoinOldAndNew(), LongValue.class, 0, 0)
-				.input1(iteration.getPartialSolution())
-				.input2(rankAggregation)
-				.name("Join Old and New")
-				.build();
-		
-		iteration.setTerminationCriterion(termination);
-		
-		FileDataSink out = new FileDataSink(new PageWithRankOutFormat(), outputPath, iteration, "Final Ranks");
-
-		Plan p = new Plan(out, "Simple PageRank");
-		p.setDefaultParallelism(parallelism);
-		return p;
-	}
-
-	@Override
-	public String getDescription() {
-		return "Parameters: <parallelism> <pages-input-path> <edges-input-path> <output-path> <max-iterations> <num-vertices> <num-dangling-vertices>";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java
deleted file mode 100644
index 70d531e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/AsciiLongArrayView.java
+++ /dev/null
@@ -1,163 +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.test.recordJobs.graph.pageRankUtil;
-
-import java.io.Serializable;
-
-import com.google.common.base.Charsets;
-
-public class AsciiLongArrayView implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private byte[] buffer;
-
-	private int offset;
-
-	private int numBytes;
-
-	private int tokenOffset;
-
-	private int tokenNumBytes;
-
-	private static final int NOT_SET = -1;
-
-	private static final int RADIX_TEN = 10;
-
-	private static final long MULTMIN_RADIX_TEN = Long.MIN_VALUE / 10;
-
-	private static final long N_MULTMAX_RADIX_TEN = -Long.MAX_VALUE / 10;
-
-	public void set(byte[] buffer, int offset, int numBytes) {
-		this.buffer = buffer;
-		this.offset = offset;
-		this.numBytes = numBytes;
-
-		this.tokenOffset = NOT_SET;
-		checkForSingleTrailingWhitespace();
-	}
-
-	private void checkForSingleTrailingWhitespace() {
-		if (Character.isWhitespace((char) buffer[offset + numBytes - 1])) {
-			numBytes--;
-		}
-	}
-
-	public int numElements() {
-		int matches = 0;
-		int pos = offset;
-		while (pos < offset + numBytes) {
-			if (Character.isWhitespace((char) buffer[pos])) {
-				matches++;
-			}
-			pos++;
-		}
-		return matches + 1;
-	}
-
-	public boolean next() {
-
-		if (tokenOffset == NOT_SET) {
-			tokenOffset = offset;
-		} else {
-			tokenOffset += tokenNumBytes + 1;
-			if (tokenOffset > offset + numBytes) {
-				return false;
-			}
-		}
-
-		tokenNumBytes = 1;
-		while (true) {
-			int candidatePos = tokenOffset + tokenNumBytes;
-			if (candidatePos >= offset + numBytes || Character.isWhitespace((char) buffer[candidatePos])) {
-				break;
-			}
-			tokenNumBytes++;
-		}
-
-		return true;
-	}
-
-	private char tokenCharAt(int pos) {
-		return (char) buffer[tokenOffset + pos];
-	}
-
-	public long element() {
-
-		long result = 0;
-		boolean negative = false;
-		int i = 0, max = tokenNumBytes;
-		long limit;
-		long multmin;
-		int digit;
-
-		if (max > 0) {
-			if (tokenCharAt(0) == '-') {
-				negative = true;
-				limit = Long.MIN_VALUE;
-				i++;
-			} else {
-				limit = -Long.MAX_VALUE;
-			}
-
-			multmin = negative ? MULTMIN_RADIX_TEN : N_MULTMAX_RADIX_TEN;
-
-			if (i < max) {
-				digit = Character.digit(tokenCharAt(i++), RADIX_TEN);
-				if (digit < 0) {
-					throw new NumberFormatException(toString());
-				} else {
-					result = -digit;
-				}
-			}
-			while (i < max) {
-				// Accumulating negatively avoids surprises near MAX_VALUE
-				digit = Character.digit(tokenCharAt(i++), RADIX_TEN);
-				if (digit < 0) {
-					throw new NumberFormatException(toString());
-				}
-				if (result < multmin) {
-					throw new NumberFormatException(toString());
-				}
-				result *= RADIX_TEN;
-				if (result < limit + digit) {
-					throw new NumberFormatException(toString());
-				}
-				result -= digit;
-			}
-		} else {
-			throw new NumberFormatException(toString());
-		}
-		if (negative) {
-			if (i > 1) {
-				return result;
-			} else { /* Only got "-" */
-				throw new NumberFormatException(toString());
-			}
-		} else {
-			return -result;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "[" + new String(buffer, offset, numBytes, Charsets.US_ASCII) + "] (buffer length: " + buffer.length +
-			", offset: " + offset + ", numBytes: " + numBytes + ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.java
deleted file mode 100644
index 4a2b360..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DanglingPageRankInputFormat.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.test.recordJobs.graph.pageRankUtil;
-
-import org.apache.flink.api.java.record.io.TextInputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.recordJobs.util.ConfigUtils;
-import org.apache.flink.types.BooleanValue;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class DanglingPageRankInputFormat extends TextInputFormat {
-	private static final long serialVersionUID = 1L;
-	
-	public static final String NUM_VERTICES_PARAMETER = "pageRank.numVertices";
-
-	private LongValue vertexID = new LongValue();
-
-	private DoubleValue initialRank;
-
-	private BooleanValue isDangling = new BooleanValue();
-
-	private AsciiLongArrayView arrayView = new AsciiLongArrayView();
-
-	private static final long DANGLING_MARKER = 1l;
-
-	@Override
-	public void configure(Configuration parameters) {
-		long numVertices = ConfigUtils.asLong(NUM_VERTICES_PARAMETER, parameters);
-		initialRank = new DoubleValue(1 / (double) numVertices);
-		super.configure(parameters);
-	}
-
-	@Override
-	public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) {
-
-		arrayView.set(bytes, offset, numBytes);
-
-		try {
-			arrayView.next();
-			vertexID.setValue(arrayView.element());
-
-			if (arrayView.next()) {
-				isDangling.set(arrayView.element() == DANGLING_MARKER);
-			} else {
-				isDangling.set(false);
-			}
-
-		} catch (NumberFormatException e) {
-			throw new RuntimeException("Error parsing " + arrayView.toString(), e);
-		}
-
-		target.clear();
-		target.addField(vertexID);
-		target.addField(initialRank);
-		target.addField(isDangling);
-
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java
deleted file mode 100644
index 70280a0..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DiffL1NormConvergenceCriterion.java
+++ /dev/null
@@ -1,44 +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.test.recordJobs.graph.pageRankUtil;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
-
-@SuppressWarnings("serial")
-public class DiffL1NormConvergenceCriterion implements ConvergenceCriterion<PageRankStats> {
-
-	private static final double EPSILON = 0.00005;
-
-	private static final Logger log = LoggerFactory.getLogger(DiffL1NormConvergenceCriterion.class);
-
-	@Override
-	public boolean isConverged(int iteration, PageRankStats pageRankStats) {
-		double diff = pageRankStats.diff();
-
-		if (log.isInfoEnabled()) {
-			log.info("Stats in iteration [" + iteration + "]: " + pageRankStats);
-			log.info("L1 norm of the vector difference is [" + diff + "] in iteration [" + iteration + "]");
-		}
-
-		return diff < EPSILON;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java
deleted file mode 100644
index d4f7a5c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductCoGroup.java
+++ /dev/null
@@ -1,129 +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.test.recordJobs.graph.pageRankUtil;
-
-import java.io.Serializable;
-import java.util.Iterator;
-
-import org.apache.flink.api.java.record.functions.CoGroupFunction;
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFieldsFirst;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.recordJobs.util.ConfigUtils;
-import org.apache.flink.types.BooleanValue;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-/**
- * In schema is_
- * INPUT = (pageId, currentRank, dangling), (pageId, partialRank).
- * OUTPUT = (pageId, newRank, dangling)
- */
-@SuppressWarnings("deprecation")
-@ConstantFieldsFirst(0)
-public class DotProductCoGroup extends CoGroupFunction implements Serializable {
-	private static final long serialVersionUID = 1L;
-	
-	public static final String NUM_VERTICES_PARAMETER = "pageRank.numVertices";
-	
-	public static final String NUM_DANGLING_VERTICES_PARAMETER = "pageRank.numDanglingVertices";
-	
-	public static final String AGGREGATOR_NAME = "pagerank.aggregator";
-	
-	private static final double BETA = 0.85;
-
-	
-	private PageRankStatsAggregator aggregator;
-
-	private long numVertices;
-
-	private long numDanglingVertices;
-
-	private double dampingFactor;
-
-	private double danglingRankFactor;
-	
-	
-	private Record accumulator = new Record();
-
-	private final DoubleValue newRank = new DoubleValue();
-
-	private BooleanValue isDangling = new BooleanValue();
-
-	private LongValue vertexID = new LongValue();
-
-	private DoubleValue doubleInstance = new DoubleValue();
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		int currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		
-		numVertices = ConfigUtils.asLong(NUM_VERTICES_PARAMETER, parameters);
-		numDanglingVertices = ConfigUtils.asLong(NUM_DANGLING_VERTICES_PARAMETER, parameters);
-
-		dampingFactor = (1d - BETA) / (double) numVertices;
-		
-		aggregator = getIterationRuntimeContext().getIterationAggregator(AGGREGATOR_NAME);
-		
-		if (currentIteration == 1) {
-			danglingRankFactor = BETA * (double) numDanglingVertices / ((double) numVertices * (double) numVertices);
-		} else {
-			PageRankStats previousAggregate = getIterationRuntimeContext().getPreviousIterationAggregate(AGGREGATOR_NAME);
-			danglingRankFactor = BETA * previousAggregate.danglingRank() / (double) numVertices;
-		}
-	}
-
-	@Override
-	public void coGroup(Iterator<Record> currentPageRankIterator, Iterator<Record> partialRanks,
-			Collector<Record> collector)
-	{
-		if (!currentPageRankIterator.hasNext()) {
-			long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue();
-			throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!");
-		}
-
-		Record currentPageRank = currentPageRankIterator.next();
-
-		long edges = 0;
-		double summedRank = 0;
-		while (partialRanks.hasNext()) {
-			summedRank += partialRanks.next().getField(1, doubleInstance).getValue();
-			edges++;
-		}
-
-		double rank = BETA * summedRank + dampingFactor + danglingRankFactor;
-		double currentRank = currentPageRank.getField(1, doubleInstance).getValue();
-		isDangling = currentPageRank.getField(2, isDangling);
-		
-		// maintain statistics to compensate for probability loss on dangling nodes
-		double danglingRankToAggregate = isDangling.get() ? rank : 0;
-		long danglingVerticesToAggregate = isDangling.get() ? 1 : 0;
-		double diff = Math.abs(currentRank - rank);
-		aggregator.aggregate(diff, rank, danglingRankToAggregate, danglingVerticesToAggregate, 1, edges);
-		
-		// return the new record
-		newRank.setValue(rank);
-		accumulator.setField(0, currentPageRank.getField(0, vertexID));
-		accumulator.setField(1, newRank);
-		accumulator.setField(2, isDangling);
-		collector.collect(accumulator);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductMatch.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductMatch.java
deleted file mode 100644
index 339cef5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/DotProductMatch.java
+++ /dev/null
@@ -1,63 +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.test.recordJobs.graph.pageRankUtil;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.record.functions.JoinFunction;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-/**
- * In schema is_
- * INPUT = (pageId, rank, dangling), (pageId, neighbors-list).
- * OUTPUT = (targetPageId, partialRank)
- */
-@SuppressWarnings("deprecation")
-public class DotProductMatch extends JoinFunction implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private Record record = new Record();
-	private LongValue vertexID = new LongValue();
-	private DoubleValue partialRank = new DoubleValue();
-	private DoubleValue rank = new DoubleValue();
-
-	private LongArrayView adjacentNeighbors = new LongArrayView();
-
-	@Override
-	public void join(Record pageWithRank, Record adjacencyList, Collector<Record> collector) throws Exception {
-
-		rank = pageWithRank.getField(1, rank);
-		adjacentNeighbors = adjacencyList.getField(1, adjacentNeighbors);
-		int numNeighbors = adjacentNeighbors.size();
-
-		double rankToDistribute = rank.getValue() / (double) numNeighbors;
-
-		partialRank.setValue(rankToDistribute);
-		record.setField(1, partialRank);
-
-		for (int n = 0; n < numNeighbors; n++) {
-			vertexID.setValue(adjacentNeighbors.getQuick(n));
-			record.setField(0, vertexID);
-			collector.collect(record);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java
deleted file mode 100644
index 6db4122..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/ImprovedAdjacencyListInputFormat.java
+++ /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.test.recordJobs.graph.pageRankUtil;
-
-import org.apache.flink.api.java.record.io.TextInputFormat;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class ImprovedAdjacencyListInputFormat extends TextInputFormat {
-	private static final long serialVersionUID = 1L;
-
-	private final LongValue vertexID = new LongValue();
-
-	private final AsciiLongArrayView arrayView = new AsciiLongArrayView();
-
-	private final LongArrayView adjacentVertices = new LongArrayView();
-
-	@Override
-	public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) {
-
-		if (numBytes == 0) {
-			return null;
-		}
-
-		arrayView.set(bytes, offset, numBytes);
-
-		int numElements = arrayView.numElements();
-		adjacentVertices.allocate(numElements - 1);
-
-		try {
-
-			int pos = 0;
-			while (arrayView.next()) {
-
-				if (pos == 0) {
-					vertexID.setValue(arrayView.element());
-				} else {
-					adjacentVertices.setQuick(pos - 1, arrayView.element());
-				}
-
-				pos++;
-			}
-
-			// sanity check
-			if (pos != numElements) {
-				throw new IllegalStateException("Should have gotten " + numElements + " elements, but saw " + pos);
-			}
-
-		} catch (RuntimeException e) {
-			throw new RuntimeException("Error parsing: " + arrayView.toString(), e);
-		}
-
-		target.clear();
-		target.addField(vertexID);
-		target.addField(adjacentVertices);
-
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/LongArrayView.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/LongArrayView.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/LongArrayView.java
deleted file mode 100644
index 770274b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/LongArrayView.java
+++ /dev/null
@@ -1,89 +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.test.recordJobs.graph.pageRankUtil;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-
-public class LongArrayView implements Value {
-	private static final long serialVersionUID = 1L;
-
-	private long[] entries = new long[0];
-
-	private int numEntries = 0;
-
-	public LongArrayView() {
-	}
-
-	public long get(int index) {
-		if (index >= numEntries) {
-			throw new ArrayIndexOutOfBoundsException();
-		}
-		return getQuick(index);
-	}
-
-	public long getQuick(int index) {
-		return entries[index];
-	}
-
-	public void allocate(int numEntries) {
-		this.numEntries = numEntries;
-		ensureCapacity();
-	}
-
-	public void set(int index, long value) {
-		if (index >= numEntries) {
-			throw new ArrayIndexOutOfBoundsException();
-		}
-		setQuick(index, value);
-	}
-
-	public void setQuick(int index, long value) {
-		entries[index] = value;
-	}
-
-	public int size() {
-		return numEntries;
-	}
-
-	private void ensureCapacity() {
-		if (entries.length < numEntries) {
-			entries = new long[numEntries];
-		}
-	}
-
-	public void write(DataOutputView out) throws IOException {
-		out.writeInt(numEntries);
-		for (int n = 0; n < numEntries; n++) {
-			out.writeLong(entries[n]);
-		}
-	}
-
-	public void read(DataInputView in) throws IOException {
-		numEntries = in.readInt();
-		ensureCapacity();
-		for (int n = 0; n < numEntries; n++) {
-			entries[n] = in.readLong();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStats.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStats.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStats.java
deleted file mode 100644
index c6d06f5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStats.java
+++ /dev/null
@@ -1,105 +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.test.recordJobs.graph.pageRankUtil;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-
-public class PageRankStats implements Value {
-	private static final long serialVersionUID = 1L;
-
-	private double diff;
-
-	private double rank;
-
-	private double danglingRank;
-
-	private long numDanglingVertices;
-
-	private long numVertices;
-
-	private long edges;
-
-	public PageRankStats() {
-	}
-
-	public PageRankStats(double diff, double rank, double danglingRank, long numDanglingVertices, long numVertices, long edges) {
-		this.diff = diff;
-		this.rank = rank;
-		this.danglingRank = danglingRank;
-		this.numDanglingVertices = numDanglingVertices;
-		this.numVertices = numVertices;
-		this.edges = edges;
-	}
-
-	public double diff() {
-		return diff;
-	}
-
-	public double rank() {
-		return rank;
-	}
-
-	public double danglingRank() {
-		return danglingRank;
-	}
-
-	public long numDanglingVertices() {
-		return numDanglingVertices;
-	}
-
-	public long numVertices() {
-		return numVertices;
-	}
-
-	public long edges() {
-		return edges;
-	}
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		out.writeDouble(diff);
-		out.writeDouble(rank);
-		out.writeDouble(danglingRank);
-		out.writeLong(numDanglingVertices);
-		out.writeLong(numVertices);
-		out.writeLong(edges);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		diff = in.readDouble();
-		rank = in.readDouble();
-		danglingRank = in.readDouble();
-		numDanglingVertices = in.readLong();
-		numVertices = in.readLong();
-		edges = in.readLong();
-	}
-
-	@Override
-	public String toString() {
-		return "PageRankStats: diff [" + diff + "], rank [" + rank + "], danglingRank [" + danglingRank +
-			"], numDanglingVertices [" + numDanglingVertices + "], numVertices [" + numVertices + "], edges [" + edges +
-			"]";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java
deleted file mode 100644
index 1457493..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageRankStatsAggregator.java
+++ /dev/null
@@ -1,73 +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.test.recordJobs.graph.pageRankUtil;
-
-import org.apache.flink.api.common.aggregators.Aggregator;
-
-@SuppressWarnings("serial")
-public class PageRankStatsAggregator implements Aggregator<PageRankStats> {
-
-	private double diff = 0;
-
-	private double rank = 0;
-
-	private double danglingRank = 0;
-
-	private long numDanglingVertices = 0;
-
-	private long numVertices = 0;
-
-	private long edges = 0;
-
-	@Override
-	public PageRankStats getAggregate() {
-		return new PageRankStats(diff, rank, danglingRank, numDanglingVertices, numVertices, edges);
-	}
-
-	public void aggregate(double diffDelta, double rankDelta, double danglingRankDelta, long danglingVerticesDelta,
-			long verticesDelta, long edgesDelta) {
-		diff += diffDelta;
-		rank += rankDelta;
-		danglingRank += danglingRankDelta;
-		numDanglingVertices += danglingVerticesDelta;
-		numVertices += verticesDelta;
-		edges += edgesDelta;
-	}
-
-	@Override
-	public void aggregate(PageRankStats pageRankStats) {
-		diff += pageRankStats.diff();
-		rank += pageRankStats.rank();
-		danglingRank += pageRankStats.danglingRank();
-		numDanglingVertices += pageRankStats.numDanglingVertices();
-		numVertices += pageRankStats.numVertices();
-		edges += pageRankStats.edges();
-	}
-
-	@Override
-	public void reset() {
-		diff = 0;
-		rank = 0;
-		danglingRank = 0;
-		numDanglingVertices = 0;
-		numVertices = 0;
-		edges = 0;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java
deleted file mode 100644
index 6c6dc42..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/graph/pageRankUtil/PageWithRankOutFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.test.recordJobs.graph.pageRankUtil;
-
-import org.apache.flink.api.java.record.io.DelimitedOutputFormat;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class PageWithRankOutFormat extends DelimitedOutputFormat {
-	private static final long serialVersionUID = 1L;
-
-	private final StringBuilder buffer = new StringBuilder();
-
-	@Override
-	public int serializeRecord(Record record, byte[] target) {
-		StringBuilder buffer = this.buffer;
-		
-		buffer.setLength(0);
-		buffer.append(record.getField(0, LongValue.class).toString());
-		buffer.append('\t');
-		buffer.append(record.getField(1, DoubleValue.class).toString());
-		buffer.append('\n');
-		
-		if (target.length < buffer.length()) {
-			return -buffer.length();
-		}
-		
-		for (int i = 0; i < buffer.length(); i++) {
-			target[i] = (byte) buffer.charAt(i);
-		}
-		return buffer.length();
-	}
-}


[5/5] flink git commit: [FLINK-1681] [tests] Remove outdated 'nephele' iteration tests.

Posted by se...@apache.org.
[FLINK-1681] [tests] Remove outdated 'nephele' iteration tests.


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

Branch: refs/heads/master
Commit: a7a57ebea6d8f60abba4fe2559af05d316112ca4
Parents: 0ba5355
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Aug 30 18:29:12 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Aug 30 22:39:17 2015 +0200

----------------------------------------------------------------------
 .../apache/flink/test/util/TestBaseUtils.java   |  45 +
 .../BulkIterationWithAllReducerITCase.java      |   6 +-
 .../test/iterative/DanglingPageRankITCase.java  | 392 ++++++++-
 .../flink/test/iterative/PageRankITCase.java    |  54 --
 .../test/iterative/nephele/ConfigUtils.java     |  64 --
 .../ConnectedComponentsNepheleITCase.java       | 837 -------------------
 .../nephele/DanglingPageRankNepheleITCase.java  |  75 --
 ...nglingPageRankWithCombinerNepheleITCase.java |  63 --
 .../IterationWithChainingNepheleITCase.java     | 296 -------
 .../test/iterative/nephele/JobGraphUtils.java   | 106 ---
 .../CustomCompensatableDanglingPageRank.java    | 315 -------
 ...mpensatableDanglingPageRankWithCombiner.java | 329 --------
 .../CustomCompensatableDotProductCoGroup.java   | 130 ---
 .../CustomCompensatableDotProductMatch.java     |  80 --
 .../CustomCompensatingMap.java                  |  82 --
 .../CustomImprovedAdjacencyListInputFormat.java |  66 --
 ...stomImprovedDanglingPageRankInputFormat.java |  66 --
 .../CustomPageWithRankOutFormat.java            |  45 -
 .../CustomRankCombiner.java                     |  57 --
 .../types/VertexWithAdjacencyList.java          |  83 --
 .../VertexWithAdjacencyListComparator.java      | 148 ----
 ...ertexWithAdjacencyListComparatorFactory.java |  39 -
 .../VertexWithAdjacencyListSerializer.java      | 112 ---
 ...ertexWithAdjacencyListSerializerFactory.java |  56 --
 .../types/VertexWithRank.java                   |  65 --
 .../types/VertexWithRankAndDangling.java        |  76 --
 .../VertexWithRankAndDanglingComparator.java    | 153 ----
 ...texWithRankAndDanglingComparatorFactory.java |  39 -
 .../VertexWithRankAndDanglingSerializer.java    |  84 --
 ...texWithRankAndDanglingSerializerFactory.java |  56 --
 .../types/VertexWithRankComparator.java         | 151 ----
 .../types/VertexWithRankComparatorFactory.java  |  39 -
 ...xWithAdjacencyListPairComparatorFactory.java |  91 --
 ...ngToVertexWithRankPairComparatorFactory.java |  91 --
 .../types/VertexWithRankSerializer.java         |  81 --
 .../types/VertexWithRankSerializerFactory.java  |  56 --
 .../danglingpagerank/AsciiLongArrayView.java    | 166 ----
 .../nephele/danglingpagerank/BooleanValue.java  |  57 --
 .../CompensatableDanglingPageRank.java          | 295 -------
 .../CompensatableDotProductCoGroup.java         | 137 ---
 .../CompensatableDotProductMatch.java           | 102 ---
 .../danglingpagerank/CompensatingMap.java       |  88 --
 .../DanglingPageGenerateRankInputFormat.java    |  62 --
 .../DiffL1NormConvergenceCriterion.java         |  44 -
 .../ImprovedAdjacencyListInputFormat.java       |  74 --
 .../ImprovedDanglingPageRankInputFormat.java    |  73 --
 .../nephele/danglingpagerank/LongArrayView.java |  88 --
 .../nephele/danglingpagerank/PageRankStats.java | 124 ---
 .../PageRankStatsAggregator.java                |  84 --
 .../danglingpagerank/PageWithRankOutFormat.java |  47 --
 .../test/recordJobs/graph/DanglingPageRank.java | 105 ---
 .../test/recordJobs/graph/SimplePageRank.java   | 194 -----
 .../graph/pageRankUtil/AsciiLongArrayView.java  | 163 ----
 .../DanglingPageRankInputFormat.java            |  78 --
 .../DiffL1NormConvergenceCriterion.java         |  44 -
 .../graph/pageRankUtil/DotProductCoGroup.java   | 129 ---
 .../graph/pageRankUtil/DotProductMatch.java     |  63 --
 .../ImprovedAdjacencyListInputFormat.java       |  76 --
 .../graph/pageRankUtil/LongArrayView.java       |  89 --
 .../graph/pageRankUtil/PageRankStats.java       | 105 ---
 .../pageRankUtil/PageRankStatsAggregator.java   |  73 --
 .../pageRankUtil/PageWithRankOutFormat.java     |  51 --
 62 files changed, 412 insertions(+), 6727 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index ce02267..87fab25 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -588,4 +588,49 @@ public class TestBaseUtils extends TestLogger {
 
 		return IOUtils.toString(is, connection.getContentEncoding() != null ? connection.getContentEncoding() : "UTF-8");
 	}
+	
+	public static class TupleComparator<T extends Tuple> implements Comparator<T> {
+
+		@Override
+		public int compare(T o1, T o2) {
+			if (o1 == null || o2 == null) {
+				throw new IllegalArgumentException("Cannot compare null tuples");
+			}
+			else if (o1.getArity() != o2.getArity()) {
+				return o1.getArity() - o2.getArity();
+			}
+			else {
+				for (int i = 0; i < o1.getArity(); i++) {
+					Object val1 = o1.getField(i);
+					Object val2 = o2.getField(i);
+					
+					int cmp;
+					if (val1 != null && val2 != null) {
+						cmp = compareValues(val1, val2);
+					}
+					else {
+						cmp = val1 == null ? (val2 == null ? 0 : -1) : 1;
+					}
+					
+					if (cmp != 0) {
+						return cmp;
+					}
+				}
+				
+				return 0;
+			}
+		}
+		
+		@SuppressWarnings("unchecked")
+		private static <X extends Comparable<X>> int compareValues(Object o1, Object o2) {
+			if (o1 instanceof Comparable && o2 instanceof Comparable) {
+				X c1 = (X) o1;
+				X c2 = (X) o2;
+				return c1.compareTo(c2);
+			}
+			else {
+				throw new IllegalArgumentException("Cannot compare tuples with non comparable elements");
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
index f4f2c18..d55a63f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java
@@ -64,10 +64,8 @@ public class BulkIterationWithAllReducerITCase extends JavaProgramTestBase {
 		
 		@Override
 		public void open(Configuration parameters) {
-			Collection<Integer> bc = getRuntimeContext().getBroadcastVariable("bc");
-			synchronized (bc) {
-				this.bcValue = bc.isEmpty() ? null : bc.iterator().next();
-			}
+			List<Integer> bc = getRuntimeContext().getBroadcastVariable("bc");
+			this.bcValue = bc.isEmpty() ? null : bc.get(0);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
index e2d095d..53496e2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java
@@ -18,35 +18,373 @@
 
 package org.apache.flink.test.iterative;
 
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.test.iterative.nephele.DanglingPageRankNepheleITCase;
-import org.apache.flink.test.recordJobs.graph.DanglingPageRank;
-import org.apache.flink.test.util.RecordAPITestBase;
+import org.apache.flink.api.common.aggregators.Aggregator;
+import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.Collector;
 
-public class DanglingPageRankITCase extends RecordAPITestBase {
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
-	protected String pagesPath;
-	protected String edgesPath;
-	protected String resultPath;
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+@RunWith(Parameterized.class)
+@SuppressWarnings({"serial", "unchecked"})
+public class DanglingPageRankITCase extends MultipleProgramsTestBase {
+
+	private static final String AGGREGATOR_NAME = "pagerank.aggregator";
+	
+	
+	public DanglingPageRankITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testDanglingPageRank() {
+		try {
+			final int NUM_ITERATIONS = 25;
+			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+			
+			DataSet<Tuple2<Long, Boolean>> vertices = env.fromElements(
+					new Tuple2<>(1L, false),
+					new Tuple2<>(2L, false),
+					new Tuple2<>(5L, false),
+					new Tuple2<>(3L, true),
+					new Tuple2<>(4L, false)
+			);
+
+			DataSet<PageWithLinks> edges = env.fromElements(
+					new PageWithLinks(2L, new long[] { 1 }),
+					new PageWithLinks(5L, new long[] { 2, 4 }),
+					new PageWithLinks(4L, new long[] { 3, 2 }),
+					new PageWithLinks(1L, new long[] { 4, 2, 3 })
+			);
+			
+			
+			final long numVertices = vertices.count();
+			final long numDanglingVertices = vertices
+					.filter(
+							new FilterFunction<Tuple2<Long, Boolean>>() {
+								@Override
+								public boolean filter(Tuple2<Long, Boolean> value) {
+									return value.f1;
+								}
+							})
+					.count();
+			
+			
+			DataSet<PageWithRankAndDangling> verticesWithInitialRank = vertices
+					.map(new MapFunction<Tuple2<Long, Boolean>, PageWithRankAndDangling>() {
+						
+						@Override
+						public PageWithRankAndDangling map(Tuple2<Long, Boolean> value) {
+							return new PageWithRankAndDangling(value.f0, 1.0 / numVertices, value.f1);
+						}
+					});
+			
+			IterativeDataSet<PageWithRankAndDangling> iteration = verticesWithInitialRank.iterate(NUM_ITERATIONS);
+
+			iteration.getAggregators().registerAggregationConvergenceCriterion(
+					AGGREGATOR_NAME,
+					new PageRankStatsAggregator(),
+					new DiffL1NormConvergenceCriterion());
+			
+			DataSet<PageWithRank> partialRanks = iteration.join(edges).where("pageId").equalTo("pageId").with(
+					new FlatJoinFunction<PageWithRankAndDangling, PageWithLinks, PageWithRank>() {
+						
+						@Override
+						public void join(PageWithRankAndDangling page,
+											PageWithLinks links,
+											Collector<PageWithRank> out)  {
+							
+							double rankToDistribute = page.rank / (double) links.targets.length;
+							PageWithRank output = new PageWithRank(0L, rankToDistribute);
+
+							for (long target : links.targets) {
+								output.pageId = target;
+								out.collect(output);
+							}
+						}
+					}
+			);
+			
+			DataSet<PageWithRankAndDangling> newRanks = 
+				iteration.coGroup(partialRanks).where("pageId").equalTo("pageId").with(
+					new RichCoGroupFunction<PageWithRankAndDangling, PageWithRank, PageWithRankAndDangling>() {
+
+						private static final double BETA = 0.85;
+
+						private final double randomJump = (1.0 - BETA) / numVertices;
+						private PageRankStatsAggregator aggregator;
+						private double danglingRankFactor;
+						
+						@Override
+						public void open(Configuration parameters) throws Exception {
+							int currentIteration = getIterationRuntimeContext().getSuperstepNumber();
+							
+							aggregator = getIterationRuntimeContext().getIterationAggregator(AGGREGATOR_NAME);
+
+							if (currentIteration == 1) {
+								danglingRankFactor = BETA * (double) numDanglingVertices / 
+										((double) numVertices * (double) numVertices);
+							} else {
+								PageRankStats previousAggregate = getIterationRuntimeContext()
+										.getPreviousIterationAggregate(AGGREGATOR_NAME);
+								danglingRankFactor = BETA * previousAggregate.danglingRank() / (double) numVertices;
+							}
+						}
+						
+						@Override
+						public void coGroup(Iterable<PageWithRankAndDangling> currentPages,
+											Iterable<PageWithRank> partialRanks,
+											Collector<PageWithRankAndDangling> out) {
+							
+							// compute the next rank
+							long edges = 0;
+							double summedRank = 0;
+							for (PageWithRank partial : partialRanks) {
+								summedRank += partial.rank;
+								edges++;
+							}
+							double rank = BETA * summedRank + randomJump + danglingRankFactor;
+							
+							// current rank, for stats and convergence
+							PageWithRankAndDangling currentPage = currentPages.iterator().next();
+							double currentRank = currentPage.rank;
+							boolean isDangling = currentPage.dangling;
+
+							// maintain statistics to compensate for probability loss on dangling nodes
+							double danglingRankToAggregate = isDangling ? rank : 0;
+							long danglingVerticesToAggregate = isDangling ? 1 : 0;
+							double diff = Math.abs(currentRank - rank);
+							aggregator.aggregate(diff, rank, danglingRankToAggregate, danglingVerticesToAggregate, 1, edges);
+
+							currentPage.rank = rank;
+							out.collect(currentPage);
+						}
+					});
+			
+			List<PageWithRankAndDangling> result = iteration.closeWith(newRanks).collect();
+			
+			double totalRank = 0.0;
+			for (PageWithRankAndDangling r : result) {
+				totalRank += r.rank;
+				assertTrue(r.pageId >= 1 && r.pageId <= 5);
+				assertTrue(r.pageId != 3 || r.dangling);
+			}
+			
+			assertEquals(1.0, totalRank, 0.001);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  custom types
+	// ------------------------------------------------------------------------
+	
+	public static class PageWithRank {
+		
+		public long pageId;
+		public double rank;
+
+		public PageWithRank() {}
+		
+		public PageWithRank(long pageId, double rank) {
+			this.pageId = pageId;
+			this.rank = rank;
+		}
+	}
+
+	public static class PageWithRankAndDangling {
+
+		public long pageId;
+		public double rank;
+		public boolean dangling; 
+
+		public PageWithRankAndDangling() {}
+
+		public PageWithRankAndDangling(long pageId, double rank, boolean dangling) {
+			this.pageId = pageId;
+			this.rank = rank;
+			this.dangling = dangling;
+		}
+
+		@Override
+		public String toString() {
+			return "PageWithRankAndDangling{" +
+					"pageId=" + pageId +
+					", rank=" + rank +
+					", dangling=" + dangling +
+					'}';
+		}
+	}
+
+	public static class PageWithLinks {
+
+		public long pageId;
+		public long[] targets;
+
+		public PageWithLinks() {}
+
+		public PageWithLinks(long pageId, long[] targets) {
+			this.pageId = pageId;
+			this.targets = targets;
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  statistics
+	// ------------------------------------------------------------------------
+
+	public static class PageRankStats implements Value {
+
+		private double diff;
+		private double rank;
+		private double danglingRank;
+		private long numDanglingVertices;
+		private long numVertices;
+		private long edges;
+
+		public PageRankStats() {}
+
+		public PageRankStats(
+					double diff, double rank, double danglingRank,
+					long numDanglingVertices, long numVertices, long edges) {
+			
+			this.diff = diff;
+			this.rank = rank;
+			this.danglingRank = danglingRank;
+			this.numDanglingVertices = numDanglingVertices;
+			this.numVertices = numVertices;
+			this.edges = edges;
+		}
+
+		public double diff() {
+			return diff;
+		}
+
+		public double rank() {
+			return rank;
+		}
+
+		public double danglingRank() {
+			return danglingRank;
+		}
+
+		public long numDanglingVertices() {
+			return numDanglingVertices;
+		}
+
+		public long numVertices() {
+			return numVertices;
+		}
+
+		public long edges() {
+			return edges;
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+			out.writeDouble(diff);
+			out.writeDouble(rank);
+			out.writeDouble(danglingRank);
+			out.writeLong(numDanglingVertices);
+			out.writeLong(numVertices);
+			out.writeLong(edges);
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+			diff = in.readDouble();
+			rank = in.readDouble();
+			danglingRank = in.readDouble();
+			numDanglingVertices = in.readLong();
+			numVertices = in.readLong();
+			edges = in.readLong();
+		}
+
+		@Override
+		public String toString() {
+			return "PageRankStats: diff [" + diff + "], rank [" + rank + "], danglingRank [" + danglingRank +
+					"], numDanglingVertices [" + numDanglingVertices + "], numVertices [" + numVertices + "], edges [" + edges +
+					"]";
+		}
+	}
 	
-	@Override
-	protected void preSubmit() throws Exception {
-		pagesPath = createTempFile("pages.txt", DanglingPageRankNepheleITCase.TEST_VERTICES);
-		edgesPath = createTempFile("edges.txt", DanglingPageRankNepheleITCase.TEST_EDGES);
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected Plan getTestJob() {
-		DanglingPageRank pr = new DanglingPageRank();
-		Plan plan = pr.getPlan(
-			String.valueOf(parallelism),
-			pagesPath,
-			edgesPath,
-			resultPath,
-			"25",	// max iterations
-			"5",	// num vertices
-			"1");	// num dangling vertices
-		return plan;
+	public static class PageRankStatsAggregator implements Aggregator<PageRankStats> {
+
+		private double diff;
+		private double rank;
+		private double danglingRank;
+		private long numDanglingVertices;
+		private long numVertices;
+		private long edges;
+
+		@Override
+		public PageRankStats getAggregate() {
+			return new PageRankStats(diff, rank, danglingRank, numDanglingVertices, numVertices, edges);
+		}
+
+		public void aggregate(double diffDelta, double rankDelta, double danglingRankDelta, long danglingVerticesDelta,
+							  long verticesDelta, long edgesDelta) {
+			diff += diffDelta;
+			rank += rankDelta;
+			danglingRank += danglingRankDelta;
+			numDanglingVertices += danglingVerticesDelta;
+			numVertices += verticesDelta;
+			edges += edgesDelta;
+		}
+
+		@Override
+		public void aggregate(PageRankStats pageRankStats) {
+			diff += pageRankStats.diff();
+			rank += pageRankStats.rank();
+			danglingRank += pageRankStats.danglingRank();
+			numDanglingVertices += pageRankStats.numDanglingVertices();
+			numVertices += pageRankStats.numVertices();
+			edges += pageRankStats.edges();
+		}
+
+		@Override
+		public void reset() {
+			diff = 0;
+			rank = 0;
+			danglingRank = 0;
+			numDanglingVertices = 0;
+			numVertices = 0;
+			edges = 0;
+		}
+	}
+
+	public static class DiffL1NormConvergenceCriterion implements ConvergenceCriterion<PageRankStats> {
+
+		private static final double EPSILON = 0.00005;
+
+		@Override
+		public boolean isConverged(int iteration, PageRankStats pageRankStats) {
+			return pageRankStats.diff() < EPSILON;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
deleted file mode 100644
index 946d89b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/PageRankITCase.java
+++ /dev/null
@@ -1,54 +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.test.iterative;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.test.recordJobs.graph.SimplePageRank;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-public class PageRankITCase extends RecordAPITestBase {
-	
-	private static final String VERTICES = "1\n2\n3\n4\n5\n6\n7\n8\n9\n10\n";
-	
-	private static final String EDGES = "1 2\n2 3\n3 4\n4 5\n5 6\n6 7\n7 8\n8 9\n9 10\n10 1\n";
-
-	protected String pagesPath;
-	protected String edgesPath;
-	protected String resultPath;
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		pagesPath = createTempFile("pages.txt", VERTICES);
-		edgesPath = createTempFile("edges.txt", EDGES);
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected Plan getTestJob() {
-		SimplePageRank pr = new SimplePageRank();
-		Plan plan = pr.getPlan(
-			String.valueOf(parallelism),
-			pagesPath,
-			edgesPath,
-			resultPath,
-			"5",	// max iterations
-			"10");	// num vertices
-		return plan;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConfigUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConfigUtils.java
deleted file mode 100644
index c9eea3d..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConfigUtils.java
+++ /dev/null
@@ -1,64 +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.test.iterative.nephele;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.flink.configuration.Configuration;
-
-public class ConfigUtils {
-
-	private ConfigUtils() {
-	}
-
-	public static int asInteger(String key, Configuration parameters) {
-		int value = parameters.getInteger(key, -1);
-		if (value == -1) {
-			throw new IllegalStateException();
-		}
-		return value;
-	}
-
-	public static double asDouble(String key, Configuration parameters) {
-		double value = Double.parseDouble(parameters.getString(key, String.valueOf(Double.NaN)));
-		if (Double.isNaN(value)) {
-			throw new IllegalStateException();
-		}
-		return value;
-	}
-
-	public static long asLong(String key, Configuration parameters) {
-		long value = parameters.getLong(key, Long.MIN_VALUE);
-		if (value == Long.MIN_VALUE) {
-			throw new IllegalStateException();
-		}
-		return value;
-	}
-
-	public static Set<Integer> asIntSet(String key, Configuration parameters) {
-		String[] tokens = parameters.getString(key, "").split(",");
-		Set<Integer> failingWorkers = new HashSet<Integer>(tokens.length);
-		for (String token : tokens) {
-			failingWorkers.add(Integer.parseInt(token));
-		}
-		return failingWorkers;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
deleted file mode 100644
index 7a3639b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ /dev/null
@@ -1,837 +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.test.iterative.nephele;
-
-import java.io.BufferedReader;
-import java.util.Collection;
-
-import org.apache.flink.api.common.aggregators.LongSumAggregator;
-import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.common.typeutils.record.RecordComparatorFactory;
-import org.apache.flink.api.common.typeutils.record.RecordPairComparatorFactory;
-import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
-import org.apache.flink.api.java.record.functions.MapFunction;
-import org.apache.flink.api.java.record.io.CsvInputFormat;
-import org.apache.flink.api.java.record.io.CsvOutputFormat;
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.api.java.record.operators.ReduceOperator.WrappingReduceFunction;
-import org.apache.flink.api.java.record.operators.ReduceOperator.WrappingClassReduceFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver;
-import org.apache.flink.runtime.operators.CollectorMapDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.GroupReduceDriver;
-import org.apache.flink.runtime.operators.JoinWithSolutionSetSecondDriver;
-import org.apache.flink.runtime.operators.chaining.ChainedCollectorMapDriver;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.MinimumComponentIDReduce;
-import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.NeighborWithComponentIDJoin;
-import org.apache.flink.test.recordJobs.graph.WorksetConnectedComponents.UpdateComponentIdMatch;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.util.RecordAPITestBase;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Tests the various variants of iteration state updates for workset iterations:
- * - unified solution set and workset tail update
- * - separate solution set and workset tail updates
- * - intermediate workset update and solution set tail
- * - intermediate solution set update and workset tail
- */
-@SuppressWarnings("deprecation")
-@RunWith(Parameterized.class)
-public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
-
-	private static final long SEED = 0xBADC0FFEEBEEFL;
-
-	private static final int NUM_VERTICES = 1000;
-
-	private static final int NUM_EDGES = 10000;
-
-	private static final int ITERATION_ID = 1;
-
-	private static final long MEM_PER_CONSUMER = 3;
-
-	private static final int parallelism = 4;
-
-	private static final double MEM_FRAC_PER_CONSUMER = (double)MEM_PER_CONSUMER/TASK_MANAGER_MEMORY_SIZE*parallelism;
-
-	protected String verticesPath;
-
-	protected String edgesPath;
-
-	protected String resultPath;
-
-	public ConnectedComponentsNepheleITCase(Configuration config) {
-		super(config);
-		setTaskManagerNumSlots(parallelism);
-	}
-
-	@Parameters
-	public static Collection<Object[]> getConfigurations() {
-		Configuration config1 = new Configuration();
-		config1.setInteger("testcase", 1);
-
-		Configuration config2 = new Configuration();
-		config2.setInteger("testcase", 2);
-
-		Configuration config3 = new Configuration();
-		config3.setInteger("testcase", 3);
-
-		Configuration config4 = new Configuration();
-		config4.setInteger("testcase", 4);
-
-		return toParameterList(config1, config2, config3, config4);
-	}
-
-	@Override
-	protected void preSubmit() throws Exception {
-		verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES));
-		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected JobGraph getJobGraph() throws Exception {
-		int maxIterations = 100;
-
-		int type = config.getInteger("testcase", 0);
-		switch (type) {
-		case 1:
-			return createJobGraphUnifiedTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations);
-		case 2:
-			return createJobGraphSeparateTails(verticesPath, edgesPath, resultPath, parallelism, maxIterations);
-		case 3:
-			return createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(verticesPath, edgesPath, resultPath, parallelism,
-				maxIterations);
-		case 4:
-			return createJobGraphSolutionSetUpdateAndWorksetTail(verticesPath, edgesPath, resultPath, parallelism,
-				maxIterations);
-		default:
-			throw new RuntimeException("Broken test configuration");
-		}
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		for (BufferedReader reader : getResultReader(resultPath)) {
-			ConnectedComponentsData.checkOddEvenResult(reader);
-		}
-	}
-
-	public static final class IdDuplicator extends MapFunction {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void map(Record record, Collector<Record> out) throws Exception {
-			record.setField(1, record.getField(0, LongValue.class));
-			out.collect(record);
-		}
-
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Invariant vertices across all variants
-	// -----------------------------------------------------------------------------------------------------------------
-
-	private static InputFormatVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
-			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
-	{
-		@SuppressWarnings("unchecked")
-		CsvInputFormat verticesInFormat = new CsvInputFormat(' ', LongValue.class);
-		InputFormatVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
-			jobGraph, numSubTasks);
-		TaskConfig verticesInputConfig = new TaskConfig(verticesInput.getConfiguration());
-		{
-			verticesInputConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			verticesInputConfig.setOutputSerializer(serializer);
-
-			// chained mapper that duplicates the id
-			TaskConfig chainedMapperConfig = new TaskConfig(new Configuration());
-			chainedMapperConfig.setStubWrapper(new UserCodeClassWrapper<IdDuplicator>(IdDuplicator.class));
-			chainedMapperConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			chainedMapperConfig.setInputLocalStrategy(0, LocalStrategy.NONE);
-			chainedMapperConfig.setInputSerializer(serializer, 0);
-
-			chainedMapperConfig.setOutputSerializer(serializer);
-			chainedMapperConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-			chainedMapperConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-			chainedMapperConfig.setOutputComparator(comparator, 0);
-			chainedMapperConfig.setOutputComparator(comparator, 1);
-
-			verticesInputConfig.addChainedTask(ChainedCollectorMapDriver.class, chainedMapperConfig, "ID Duplicator");
-		}
-
-		return verticesInput;
-	}
-
-	private static InputFormatVertex createEdgesInput(JobGraph jobGraph, String edgesPath, int numSubTasks,
-			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
-	{
-		// edges
-		@SuppressWarnings("unchecked")
-		CsvInputFormat edgesInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class);
-		InputFormatVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
-			numSubTasks);
-		TaskConfig edgesInputConfig = new TaskConfig(edgesInput.getConfiguration());
-		{
-			edgesInputConfig.setOutputSerializer(serializer);
-			edgesInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-			edgesInputConfig.setOutputComparator(comparator, 0);
-		}
-
-		return edgesInput;
-	}
-
-	private static JobVertex createIterationHead(JobGraph jobGraph, int numSubTasks,
-			TypeSerializerFactory<?> serializer,
-			TypeComparatorFactory<?> comparator,
-			TypePairComparatorFactory<?, ?> pairComparator) {
-
-		JobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Join With Edges (Iteration Head)", jobGraph, numSubTasks);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		{
-			headConfig.setIterationId(ITERATION_ID);
-
-			// initial input / workset
-			headConfig.addInputToGroup(0);
-			headConfig.setInputSerializer(serializer, 0);
-			headConfig.setInputComparator(comparator, 0);
-			headConfig.setInputLocalStrategy(0, LocalStrategy.NONE);
-			headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
-
-			// regular plan input (second input to the join)
-			headConfig.addInputToGroup(1);
-			headConfig.setInputSerializer(serializer, 1);
-			headConfig.setInputComparator(comparator, 1);
-			headConfig.setInputLocalStrategy(1, LocalStrategy.NONE);
-			headConfig.setInputCached(1, true);
-			headConfig.setRelativeInputMaterializationMemory(1, MEM_FRAC_PER_CONSUMER);
-
-			// initial solution set input
-			headConfig.addInputToGroup(2);
-			headConfig.setInputSerializer(serializer, 2);
-			headConfig.setInputComparator(comparator, 2);
-			headConfig.setInputLocalStrategy(2, LocalStrategy.NONE);
-			headConfig.setIterationHeadSolutionSetInputIndex(2);
-
-			headConfig.setSolutionSetSerializer(serializer);
-			headConfig.setSolutionSetComparator(comparator);
-
-			// back channel / iterations
-			headConfig.setIsWorksetIteration();
-			headConfig.setRelativeBackChannelMemory(MEM_FRAC_PER_CONSUMER);
-			headConfig.setRelativeSolutionSetMemory(MEM_FRAC_PER_CONSUMER );
-
-			// output into iteration
-			headConfig.setOutputSerializer(serializer);
-			headConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-			headConfig.setOutputComparator(comparator, 0);
-
-			// final output
-			TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
-			headFinalOutConfig.setOutputSerializer(serializer);
-			headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);
-
-			// the sync
-			headConfig.setIterationHeadIndexOfSyncOutput(2);
-
-			// the driver
-			headConfig.setDriver(BuildSecondCachedJoinDriver.class);
-			headConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-			headConfig.setStubWrapper(
-				new UserCodeClassWrapper<NeighborWithComponentIDJoin>(NeighborWithComponentIDJoin.class));
-			headConfig.setDriverComparator(comparator, 0);
-			headConfig.setDriverComparator(comparator, 1);
-			headConfig.setDriverPairComparator(pairComparator);
-			headConfig.setRelativeMemoryDriver(MEM_FRAC_PER_CONSUMER);
-
-			headConfig.addIterationAggregator(
-				WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME, new LongSumAggregator());
-		}
-
-		return head;
-	}
-
-	private static JobVertex createIterationIntermediate(JobGraph jobGraph, int numSubTasks,
-			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
-	{
-		// --------------- the intermediate (reduce to min id) ---------------
-		JobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Find Min Component-ID", jobGraph, numSubTasks);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-		{
-			intermediateConfig.setIterationId(ITERATION_ID);
-
-			intermediateConfig.addInputToGroup(0);
-			intermediateConfig.setInputSerializer(serializer, 0);
-			intermediateConfig.setInputComparator(comparator, 0);
-			intermediateConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-			intermediateConfig.setRelativeMemoryInput(0, MEM_FRAC_PER_CONSUMER);
-			intermediateConfig.setFilehandlesInput(0, 64);
-			intermediateConfig.setSpillingThresholdInput(0, 0.85f);
-
-			intermediateConfig.setOutputSerializer(serializer);
-			intermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-
-			intermediateConfig.setDriver(GroupReduceDriver.class);
-			intermediateConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE);
-			intermediateConfig.setDriverComparator(comparator, 0);
-			intermediateConfig.setStubWrapper(
-				new UserCodeObjectWrapper<WrappingReduceFunction>(new WrappingClassReduceFunction(MinimumComponentIDReduce.class)));
-		}
-
-		return intermediate;
-	}
-
-	private static OutputFormatVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
-			TypeSerializerFactory<?> serializer) {
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
-		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
-		{
-
-			outputConfig.addInputToGroup(0);
-			outputConfig.setInputSerializer(serializer, 0);
-
-			outputConfig.setStubWrapper(new UserCodeClassWrapper<CsvOutputFormat>(CsvOutputFormat.class));
-			outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, resultPath);
-
-			Configuration outputUserConfig = outputConfig.getStubParameters();
-			outputUserConfig.setString(CsvOutputFormat.RECORD_DELIMITER_PARAMETER, "\n");
-			outputUserConfig.setString(CsvOutputFormat.FIELD_DELIMITER_PARAMETER, " ");
-			outputUserConfig.setClass(CsvOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 0, LongValue.class);
-			outputUserConfig.setInteger(CsvOutputFormat.RECORD_POSITION_PARAMETER_PREFIX + 0, 0);
-			outputUserConfig.setClass(CsvOutputFormat.FIELD_TYPE_PARAMETER_PREFIX + 1, LongValue.class);
-			outputUserConfig.setInteger(CsvOutputFormat.RECORD_POSITION_PARAMETER_PREFIX + 1, 1);
-			outputUserConfig.setInteger(CsvOutputFormat.NUM_FIELDS_PARAMETER, 2);
-		}
-
-		return output;
-	}
-
-	private static JobVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
-		JobVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setNumberOfIterations(maxIterations);
-		syncConfig.setIterationId(ITERATION_ID);
-		syncConfig.addIterationAggregator(WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME,
-			new LongSumAggregator());
-		syncConfig.setConvergenceCriterion(WorksetEmptyConvergenceCriterion.AGGREGATOR_NAME,
-			new WorksetEmptyConvergenceCriterion());
-
-		return sync;
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Unified solution set and workset tail update
-	// -----------------------------------------------------------------------------------------------------------------
-
-	public JobGraph createJobGraphUnifiedTails(
-			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-	{
-		// -- init -------------------------------------------------------------------------------------------------
-		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
-		@SuppressWarnings("unchecked")
-		final TypeComparatorFactory<?> comparator =
-			new RecordComparatorFactory(new int[] { 0 }, new Class[] { LongValue.class }, new boolean[] { true });
-		final TypePairComparatorFactory<?, ?> pairComparator = RecordPairComparatorFactory.get();
-
-		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
-
-		// -- invariant vertices -----------------------------------------------------------------------------------
-		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
-		JobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
-
-		JobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-
-		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
-
-		// --------------- the tail (solution set join) ---------------
-		JobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph, numSubTasks);
-		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
-		{
-			tailConfig.setIterationId(ITERATION_ID);
-
-			tailConfig.setIsWorksetIteration();
-			tailConfig.setIsWorksetUpdate();
-
-			tailConfig.setIsSolutionSetUpdate();
-			tailConfig.setIsSolutionSetUpdateWithoutReprobe();
-
-			// inputs and driver
-			tailConfig.addInputToGroup(0);
-			tailConfig.setInputSerializer(serializer, 0);
-
-			// output
-			tailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			tailConfig.setDriver(JoinWithSolutionSetSecondDriver.class);
-			tailConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-			tailConfig.setDriverComparator(comparator, 0);
-			tailConfig.setDriverPairComparator(pairComparator);
-			
-			tailConfig.setStubWrapper(new UserCodeClassWrapper<UpdateComponentIdMatch>(UpdateComponentIdMatch.class));
-		}
-
-		// -- edges ------------------------------------------------------------------------------------------------
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(edges, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.ALL_TO_ALL);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
-
-		JobGraphUtils.connect(intermediate, tail, DistributionPattern.POINTWISE);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		vertices.setSlotSharingGroup(sharingGroup);
-		edges.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		tail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-		
-		intermediate.setStrictlyCoLocatedWith(head);
-		tail.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-
-	public JobGraph createJobGraphSeparateTails(
-			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-	{
-		// -- init -------------------------------------------------------------------------------------------------
-		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
-		@SuppressWarnings("unchecked")
-		final TypeComparatorFactory<?> comparator =
-			new RecordComparatorFactory(new int[] { 0 }, new Class[] { LongValue.class }, new boolean[] { true });
-		final TypePairComparatorFactory<?, ?> pairComparator = RecordPairComparatorFactory.get();
-
-		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
-
-		// input
-		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
-
-		// head
-		JobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		headConfig.setWaitForSolutionSetUpdate();
-
-		// intermediate
-		JobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-
-		// output and auxiliaries
-		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
-
-		// ------------------ the intermediate (ss join) ----------------------
-		JobVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Solution Set Join", jobGraph, numSubTasks);
-		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
-		{
-			ssJoinIntermediateConfig.setIterationId(ITERATION_ID);
-
-			// inputs
-			ssJoinIntermediateConfig.addInputToGroup(0);
-			ssJoinIntermediateConfig.setInputSerializer(serializer, 0);
-
-			// output
-			ssJoinIntermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			ssJoinIntermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			ssJoinIntermediateConfig.setOutputComparator(comparator, 0);
-			ssJoinIntermediateConfig.setOutputComparator(comparator, 1);
-
-			ssJoinIntermediateConfig.setOutputSerializer(serializer);
-
-			// driver
-			ssJoinIntermediateConfig.setDriver(JoinWithSolutionSetSecondDriver.class);
-			ssJoinIntermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-			ssJoinIntermediateConfig.setDriverComparator(comparator, 0);
-			ssJoinIntermediateConfig.setDriverPairComparator(pairComparator);
-			
-			ssJoinIntermediateConfig.setStubWrapper(
-				new UserCodeClassWrapper<UpdateComponentIdMatch>(UpdateComponentIdMatch.class));
-		}
-
-		// -------------------------- ss tail --------------------------------
-		JobVertex ssTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail",
-			jobGraph, numSubTasks);
-		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
-		{
-			ssTailConfig.setIterationId(ITERATION_ID);
-			ssTailConfig.setIsSolutionSetUpdate();
-			ssTailConfig.setIsWorksetIteration();
-
-			// inputs and driver
-			ssTailConfig.addInputToGroup(0);
-			ssTailConfig.setInputSerializer(serializer, 0);
-			ssTailConfig.setInputAsynchronouslyMaterialized(0, true);
-			ssTailConfig.setRelativeInputMaterializationMemory(0, MEM_FRAC_PER_CONSUMER);
-
-			// output
-			ssTailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			ssTailConfig.setDriver(CollectorMapDriver.class);
-			ssTailConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			ssTailConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
-		}
-
-		// -------------------------- ws tail --------------------------------
-		JobVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
-			jobGraph, numSubTasks);
-		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
-		{
-			wsTailConfig.setIterationId(ITERATION_ID);
-			wsTailConfig.setIsWorksetIteration();
-			wsTailConfig.setIsWorksetUpdate();
-
-			// inputs and driver
-			wsTailConfig.addInputToGroup(0);
-			wsTailConfig.setInputSerializer(serializer, 0);
-
-			// output
-			wsTailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			wsTailConfig.setDriver(CollectorMapDriver.class);
-			wsTailConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			wsTailConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
-		}
-
-		// --------------- the wiring ---------------------
-
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(edges, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.ALL_TO_ALL);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
-
-		JobGraphUtils.connect(intermediate, ssJoinIntermediate, DistributionPattern.POINTWISE);
-		ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(ssJoinIntermediate, ssTail, DistributionPattern.POINTWISE);
-		ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(ssJoinIntermediate, wsTail, DistributionPattern.POINTWISE);
-		wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		vertices.setSlotSharingGroup(sharingGroup);
-		edges.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		ssJoinIntermediate.setSlotSharingGroup(sharingGroup);
-		wsTail.setSlotSharingGroup(sharingGroup);
-		ssTail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-		
-		intermediate.setStrictlyCoLocatedWith(head);
-		ssJoinIntermediate.setStrictlyCoLocatedWith(head);
-		wsTail.setStrictlyCoLocatedWith(head);
-		ssTail.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-
-	public JobGraph createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(
-			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-	{
-		// -- init -------------------------------------------------------------------------------------------------
-		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
-		@SuppressWarnings("unchecked")
-		final TypeComparatorFactory<?> comparator =
-			new RecordComparatorFactory(new int[] { 0 }, new Class[] { LongValue.class }, new boolean[] { true });
-		final TypePairComparatorFactory<?, ?> pairComparator = RecordPairComparatorFactory.get();
-
-		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Workset Update, Solution Set Tail)");
-
-		// input
-		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
-
-		// head
-		JobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		headConfig.setWaitForSolutionSetUpdate();
-
-		// intermediate
-		JobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-
-		// output and auxiliaries
-		JobVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
-
-		// ------------------ the intermediate (ws update) ----------------------
-		JobVertex wsUpdateIntermediate =
-			JobGraphUtils.createTask(IterationIntermediatePactTask.class, "WorksetUpdate", jobGraph, numSubTasks);
-		TaskConfig wsUpdateConfig = new TaskConfig(wsUpdateIntermediate.getConfiguration());
-		{
-			wsUpdateConfig.setIterationId(ITERATION_ID);
-			wsUpdateConfig.setIsWorksetIteration();
-			wsUpdateConfig.setIsWorksetUpdate();
-
-			// inputs
-			wsUpdateConfig.addInputToGroup(0);
-			wsUpdateConfig.setInputSerializer(serializer, 0);
-
-			// output
-			wsUpdateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			wsUpdateConfig.setOutputComparator(comparator, 0);
-
-			wsUpdateConfig.setOutputSerializer(serializer);
-
-			// driver
-			wsUpdateConfig.setDriver(JoinWithSolutionSetSecondDriver.class);
-			wsUpdateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-			wsUpdateConfig.setDriverComparator(comparator, 0);
-			wsUpdateConfig.setDriverPairComparator(pairComparator);
-			
-			wsUpdateConfig.setStubWrapper(new UserCodeClassWrapper<UpdateComponentIdMatch>(
-				UpdateComponentIdMatch.class));
-		}
-
-		// -------------------------- ss tail --------------------------------
-		JobVertex ssTail =
-			JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail", jobGraph, numSubTasks);
-		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
-		{
-			ssTailConfig.setIterationId(ITERATION_ID);
-			ssTailConfig.setIsSolutionSetUpdate();
-			ssTailConfig.setIsWorksetIteration();
-
-			// inputs and driver
-			ssTailConfig.addInputToGroup(0);
-			ssTailConfig.setInputSerializer(serializer, 0);
-
-			// output
-			ssTailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			ssTailConfig.setDriver(CollectorMapDriver.class);
-			ssTailConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			ssTailConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
-		}
-
-		// edges
-
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(edges, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.ALL_TO_ALL);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
-
-		JobGraphUtils.connect(intermediate, wsUpdateIntermediate,
-				DistributionPattern.POINTWISE);
-		wsUpdateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(wsUpdateIntermediate, ssTail, DistributionPattern.POINTWISE);
-		ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		vertices.setSlotSharingGroup(sharingGroup);
-		edges.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		wsUpdateIntermediate.setSlotSharingGroup(sharingGroup);
-		ssTail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-
-		intermediate.setStrictlyCoLocatedWith(head);
-		wsUpdateIntermediate.setStrictlyCoLocatedWith(head);
-		ssTail.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	// Intermediate solution set update and workset tail
-	// -----------------------------------------------------------------------------------------------------------------
-
-	public JobGraph createJobGraphSolutionSetUpdateAndWorksetTail(
-			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-	{
-		// -- init -------------------------------------------------------------------------------------------------
-		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
-		@SuppressWarnings("unchecked")
-		final TypeComparatorFactory<?> comparator =
-			new RecordComparatorFactory(new int[] { 0 }, new Class[] { LongValue.class }, new boolean[] { true });
-		final TypePairComparatorFactory<?, ?> pairComparator = RecordPairComparatorFactory.get();
-
-		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Solution Set Update, Workset Tail)");
-
-		// input
-		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
-
-		// head
-		JobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
-
-		// intermediate
-		JobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-
-		// output and auxiliaries
-		JobVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
-
-		// ------------------ the intermediate (ss update) ----------------------
-		JobVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"Solution Set Update", jobGraph, numSubTasks);
-		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
-		{
-			ssJoinIntermediateConfig.setIterationId(ITERATION_ID);
-			ssJoinIntermediateConfig.setIsSolutionSetUpdate();
-			ssJoinIntermediateConfig.setIsSolutionSetUpdateWithoutReprobe();
-
-			// inputs
-			ssJoinIntermediateConfig.addInputToGroup(0);
-			ssJoinIntermediateConfig.setInputSerializer(serializer, 0);
-
-			// output
-			ssJoinIntermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			ssJoinIntermediateConfig.setOutputComparator(comparator, 0);
-
-			ssJoinIntermediateConfig.setOutputSerializer(serializer);
-
-			// driver
-			ssJoinIntermediateConfig.setDriver(JoinWithSolutionSetSecondDriver.class);
-			ssJoinIntermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-			ssJoinIntermediateConfig.setDriverComparator(comparator, 0);
-			ssJoinIntermediateConfig.setDriverPairComparator(pairComparator);
-			
-			ssJoinIntermediateConfig.setStubWrapper(new UserCodeClassWrapper<UpdateComponentIdMatch>(UpdateComponentIdMatch.class));
-		}
-
-		// -------------------------- ws tail --------------------------------
-		JobVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail", jobGraph, numSubTasks);
-		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
-		{
-			wsTailConfig.setIterationId(ITERATION_ID);
-			wsTailConfig.setIsWorksetIteration();
-			wsTailConfig.setIsWorksetUpdate();
-
-			// inputs and driver
-			wsTailConfig.addInputToGroup(0);
-			wsTailConfig.setInputSerializer(serializer, 0);
-
-			// output
-			wsTailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			wsTailConfig.setDriver(CollectorMapDriver.class);
-			wsTailConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			wsTailConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
-		}
-
-		// --------------- the wiring ---------------------
-
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(edges, head, DistributionPattern.ALL_TO_ALL);
-		JobGraphUtils.connect(vertices, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.ALL_TO_ALL);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
-
-		JobGraphUtils.connect(intermediate, ssJoinIntermediate, DistributionPattern.POINTWISE);
-		ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(ssJoinIntermediate, wsTail, DistributionPattern.POINTWISE);
-		wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-
-		
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		vertices.setSlotSharingGroup(sharingGroup);
-		edges.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		ssJoinIntermediate.setSlotSharingGroup(sharingGroup);
-		wsTail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-
-		intermediate.setStrictlyCoLocatedWith(head);
-		ssJoinIntermediate.setStrictlyCoLocatedWith(head);
-		wsTail.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-
-	public static final class DummyMapper extends MapFunction {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void map(Record rec, Collector<Record> out) {
-			out.collect(rec);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.java
deleted file mode 100644
index 516309c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankNepheleITCase.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.test.iterative.nephele;
-
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.CustomCompensatableDanglingPageRank;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-public class DanglingPageRankNepheleITCase extends RecordAPITestBase {
-	
-	public static final String TEST_VERTICES = "1\n" +
-	                                           "2\n" +
-	                                           "5\n" +
-	                                           "3 1\n" +
-	                                           "4";
-
-	public static final String TEST_EDGES = "2 1\n" +
-	                                        "5 2 4\n" +
-	                                        "4 3 2\n" +
-	                                        "1 4 2 3";
-	
-	protected String pagesWithRankPath;
-	protected String edgesPath;
-	protected String resultPath;
-
-	public DanglingPageRankNepheleITCase(){
-		setTaskManagerNumSlots(parallelism);
-	}
-
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		this.pagesWithRankPath = createTempFile("pagesWithRank", TEST_VERTICES);
-		this.edgesPath = createTempFile("edges", TEST_EDGES);
-		this.resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected JobGraph getJobGraph() throws Exception {
-		String[] parameters = new String[] {
-			Integer.valueOf(parallelism).toString(),
-			pagesWithRankPath,
-			edgesPath,
-			resultPath,
-			"<none>",
-			"2",
-			"5",
-			"5",
-			"30",
-			"5",
-			"1",
-			"0",
-			"100",
-			"0"
-		};
-		
-		return CustomCompensatableDanglingPageRank.getJobGraph(parameters);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
deleted file mode 100644
index ba22ce5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/DanglingPageRankWithCombinerNepheleITCase.java
+++ /dev/null
@@ -1,63 +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.test.iterative.nephele;
-
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.CustomCompensatableDanglingPageRankWithCombiner;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-public class DanglingPageRankWithCombinerNepheleITCase extends RecordAPITestBase {
-	
-	protected String pagesWithRankPath;
-	protected String edgesPath;
-	protected String resultPath;
-
-	public DanglingPageRankWithCombinerNepheleITCase(){
-		setTaskManagerNumSlots(parallelism);
-	}
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		this.pagesWithRankPath = createTempFile("pagesWithRank", DanglingPageRankNepheleITCase.TEST_VERTICES);
-		this.edgesPath = createTempFile("edges", DanglingPageRankNepheleITCase.TEST_EDGES);
-		this.resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected JobGraph getJobGraph() throws Exception {
-		String[] parameters = new String[] {
-			Integer.valueOf(parallelism).toString(),
-			pagesWithRankPath,
-			edgesPath,
-			resultPath,
-			"<none>",
-			"2",
-			"5",
-			"3",
-			"30",
-			"5",
-			"1",
-			"0",
-			"100",
-			"0"
-		};
-		
-		return CustomCompensatableDanglingPageRankWithCombiner.getJobGraph(parameters);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
deleted file mode 100644
index 7a3135c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ /dev/null
@@ -1,296 +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.test.iterative.nephele;
-
-import java.util.Collection;
-
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.common.typeutils.record.RecordComparatorFactory;
-import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
-import org.apache.flink.api.java.record.functions.MapFunction;
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.operators.CollectorMapDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.GroupReduceDriver;
-import org.apache.flink.runtime.operators.chaining.ChainedCollectorMapDriver;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.test.iterative.IterationWithChainingITCase;
-import org.apache.flink.test.recordJobs.kmeans.udfs.CoordVector;
-import org.apache.flink.test.recordJobs.kmeans.udfs.PointInFormat;
-import org.apache.flink.test.recordJobs.kmeans.udfs.PointOutFormat;
-import org.apache.flink.test.util.RecordAPITestBase;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-/**
- * Tests chained iteration tails.
- * <p>
- * GitHub issue #123 reports a problem with chaining of tasks to iteration tails. The initial fix worked around the
- * issue by having the compiler *not* chain tasks to an iteration tail. The existing IterationWithChainingITCase only
- * tests this compiler behavior. The JobGraph and bypasses the compiler to test the original chaining problem.
- * <p>
- * A chained mapper after the iteration tail (dummy reduce) increments the given input points in each iteration. The
- * final result will only be correct, if the chained mapper is successfully executed.
- * 
- * {@link IterationWithChainingITCase}
- */
-@SuppressWarnings("deprecation")
-@RunWith(Parameterized.class)
-public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
-
-	private static final String INPUT_STRING = "0|%d.25|\n" + "1|%d.25|\n";
-
-	private String dataPath;
-
-	private String resultPath;
-
-	public IterationWithChainingNepheleITCase(Configuration config) {
-		super(config);
-		setTaskManagerNumSlots(parallelism);
-	}
-
-	@Override
-	protected void preSubmit() throws Exception {
-		String initialInput = String.format(INPUT_STRING, 1, 2);
-		dataPath = createTempFile("data_points.txt", initialInput);
-		resultPath = getTempFilePath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		int maxIterations = config.getInteger("ChainedMapperNepheleITCase#MaxIterations", 1);
-		String result = String.format(INPUT_STRING, 1 + maxIterations, 2 + maxIterations);
-		compareResultsByLinesInMemory(result, resultPath);
-	}
-
-	@Parameterized.Parameters
-	public static Collection<Object[]> getConfigurations() {
-		Configuration config = new Configuration();
-		config.setInteger("ChainedMapperNepheleITCase#NoSubtasks", parallelism);
-		config.setInteger("ChainedMapperNepheleITCase#MaxIterations", 2);
-		return toParameterList(config);
-	}
-
-	@Override
-	protected JobGraph getJobGraph() throws Exception {
-		int numSubTasks = config.getInteger("ChainedMapperNepheleITCase#NoSubtasks", 1);
-		int maxIterations = config.getInteger("ChainedMapperNepheleITCase#MaxIterations", 1);
-
-		return getTestJobGraph(dataPath, resultPath, numSubTasks, maxIterations);
-	}
-
-	private JobGraph getTestJobGraph(String inputPath, String outputPath, int numSubTasks, int maxIterations) {
-
-		final JobGraph jobGraph = new JobGraph("Iteration Tail with Chaining");
-
-		final TypeSerializerFactory<Record> serializer = RecordSerializerFactory.get();
-
-		@SuppressWarnings("unchecked")
-		final TypeComparatorFactory<Record> comparator =
-			new RecordComparatorFactory(new int[] { 0 }, new Class[] { IntValue.class });
-
-		final int ITERATION_ID = 1;
-
-		// --------------------------------------------------------------------------------------------------------------
-		// 1. VERTICES
-		// --------------------------------------------------------------------------------------------------------------
-
-		// - input -----------------------------------------------------------------------------------------------------
-		InputFormatVertex input = JobGraphUtils.createInput(
-			new PointInFormat(), inputPath, "Input", jobGraph, numSubTasks);
-		TaskConfig inputConfig = new TaskConfig(input.getConfiguration());
-		{
-			inputConfig.setOutputSerializer(serializer);
-			inputConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		}
-
-		// - head ------------------------------------------------------------------------------------------------------
-		JobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		{
-			headConfig.setIterationId(ITERATION_ID);
-
-			// input to iteration head
-			headConfig.addInputToGroup(0);
-			headConfig.setInputSerializer(serializer, 0);
-			headConfig.setInputLocalStrategy(0, LocalStrategy.NONE);
-			headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
-
-			// output into iteration
-			headConfig.setOutputSerializer(serializer);
-			headConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-			headConfig.setOutputComparator(comparator, 0);
-
-			// final output
-			TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
-			headFinalOutConfig.setOutputSerializer(serializer);
-			headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);
-
-			// the sync
-			headConfig.setIterationHeadIndexOfSyncOutput(2);
-
-			// driver
-			headConfig.setDriver(CollectorMapDriver.class);
-			headConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			headConfig.setStubWrapper(new UserCodeClassWrapper<DummyMapper>(DummyMapper.class));
-
-			// back channel
-			headConfig.setRelativeBackChannelMemory(1.0);
-		}
-
-		// - tail ------------------------------------------------------------------------------------------------------
-		JobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Chained Iteration Tail", jobGraph, numSubTasks);
-		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
-		{
-			tailConfig.setIterationId(ITERATION_ID);
-
-			// inputs and driver
-			tailConfig.addInputToGroup(0);
-			tailConfig.setInputSerializer(serializer, 0);
-
-			// output
-			tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-			tailConfig.setOutputSerializer(serializer);
-
-			// the driver
-			tailConfig.setDriver(GroupReduceDriver.class);
-			tailConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE);
-			tailConfig.setDriverComparator(comparator, 0);
-			tailConfig.setStubWrapper(new UserCodeClassWrapper<DummyReducer>(DummyReducer.class));
-
-			// chained mapper
-			TaskConfig chainedMapperConfig = new TaskConfig(new Configuration());
-			chainedMapperConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-			chainedMapperConfig.setStubWrapper(new UserCodeClassWrapper<IncrementCoordinatesMapper>(
-				IncrementCoordinatesMapper.class));
-
-			chainedMapperConfig.setInputLocalStrategy(0, LocalStrategy.NONE);
-			chainedMapperConfig.setInputSerializer(serializer, 0);
-
-			chainedMapperConfig.setOutputSerializer(serializer);
-
-			chainedMapperConfig.setIsWorksetUpdate();
-
-			tailConfig.addChainedTask(ChainedCollectorMapDriver.class, chainedMapperConfig, "Chained ID Mapper");
-		}
-
-		// - output ----------------------------------------------------------------------------------------------------
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
-		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
-		{
-			outputConfig.addInputToGroup(0);
-			outputConfig.setInputSerializer(serializer, 0);
-
-			outputConfig.setStubWrapper(new UserCodeClassWrapper<PointOutFormat>(PointOutFormat.class));
-			outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath);
-		}
-
-		// - sync ------------------------------------------------------------------------------------------------------
-		JobVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setNumberOfIterations(maxIterations);
-		syncConfig.setIterationId(ITERATION_ID);
-
-		// --------------------------------------------------------------------------------------------------------------
-		// 2. EDGES
-		// --------------------------------------------------------------------------------------------------------------
-		JobGraphUtils.connect(input, head, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, tail, DistributionPattern.ALL_TO_ALL);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-
-		// --------------------------------------------------------------------------------------------------------------
-		// 3. INSTANCE SHARING
-		// --------------------------------------------------------------------------------------------------------------
-		
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		
-		input.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		tail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-		
-		tail.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-
-	public static final class DummyMapper extends MapFunction {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void map(Record rec, Collector<Record> out) {
-			out.collect(rec);
-		}
-	}
-
-	public static final class DummyReducer implements GroupReduceFunction<Record, Record> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Record> it, Collector<Record> out) {
-			for (Record r :it) {
-				out.collect(r);
-			}
-		}
-	}
-
-	public static final class IncrementCoordinatesMapper extends MapFunction {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void map(Record rec, Collector<Record> out) {
-			CoordVector coord = rec.getField(1, CoordVector.class);
-
-			double[] vector = coord.getCoordinates();
-			for (int i = 0; i < vector.length; i++) {
-				vector[i]++;
-			}
-
-			rec.setField(1, coord);
-			out.collect(rec);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
deleted file mode 100644
index 4edc83e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.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.test.iterative.nephele;
-
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
-import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.operators.DataSinkTask;
-import org.apache.flink.runtime.operators.DataSourceTask;
-import org.apache.flink.runtime.operators.RegularPactTask;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-
-public class JobGraphUtils {
-
-	public static final long MEGABYTE = 1024l * 1024l;
-
-	private JobGraphUtils() {}
-	
-	public static <T extends FileInputFormat<?>> InputFormatVertex createInput(T stub, String path, String name, JobGraph graph,
-			int parallelism)
-	{
-		stub.setFilePath(path);
-		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, parallelism);
-	}
-
-	private static <T extends InputFormat<?,?>> InputFormatVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
-			int parallelism)
-	{
-		InputFormatVertex inputVertex = new InputFormatVertex(name);
-		graph.addVertex(inputVertex);
-		
-		inputVertex.setInvokableClass(DataSourceTask.class);
-		inputVertex.setParallelism(parallelism);
-
-		TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration());
-		inputConfig.setStubWrapper(stub);
-		
-		return inputVertex;
-	}
-
-//	public static void connect(AbstractJobVertex source, AbstractJobVertex target, ChannelType channelType,
-//			DistributionPattern distributionPattern, ShipStrategyType shipStrategy) throws JobGraphDefinitionException
-//	{
-//		source.connectTo(target, channelType, CompressionLevel.NO_COMPRESSION, distributionPattern);
-//		new TaskConfig(source.getConfiguration()).addOutputShipStrategy(shipStrategy);
-//	}
-	
-	public static void connect(JobVertex source, JobVertex target, DistributionPattern distributionPattern) {
-		target.connectNewDataSetAsInput(source, distributionPattern);
-	}
-
-	@SuppressWarnings("rawtypes") 
-	public static JobVertex createTask(Class<? extends RegularPactTask> task, String name, JobGraph graph, int parallelism)
-	{
-		JobVertex taskVertex = new JobVertex(name);
-		graph.addVertex(taskVertex);
-		
-		taskVertex.setInvokableClass(task);
-		taskVertex.setParallelism(parallelism);
-		return taskVertex;
-	}
-
-	public static JobVertex createSync(JobGraph jobGraph, int parallelism) {
-		JobVertex sync = new JobVertex("BulkIterationSync");
-		jobGraph.addVertex(sync);
-		
-		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
-		sync.setParallelism(1);
-		
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, parallelism);
-		return sync;
-	}
-
-	public static OutputFormatVertex createFileOutput(JobGraph jobGraph, String name, int parallelism) {
-		OutputFormatVertex sinkVertex = new OutputFormatVertex(name);
-		jobGraph.addVertex(sinkVertex);
-		
-		sinkVertex.setInvokableClass(DataSinkTask.class);
-		sinkVertex.setParallelism(parallelism);
-		return sinkVertex;
-	}
-}


[3/5] flink git commit: [FLINK-1681] [tests] Remove outdated 'nephele' iteration tests.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
deleted file mode 100644
index d83c3fb..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparator.java
+++ /dev/null
@@ -1,153 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.base.LongComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-public final class VertexWithRankAndDanglingComparator extends TypeComparator<VertexWithRankAndDangling> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private long reference;
-
-	@SuppressWarnings("rawtypes")
-	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
-
-	@Override
-	public int hash(VertexWithRankAndDangling record) {
-		final long value = record.getVertexID();
-		return 43 + (int) (value ^ value >>> 32);
-	}
-
-	@Override
-	public void setReference(VertexWithRankAndDangling toCompare) {
-		this.reference = toCompare.getVertexID();
-	}
-
-	@Override
-	public boolean equalToReference(VertexWithRankAndDangling candidate) {
-		return candidate.getVertexID() == this.reference;
-	}
-
-	@Override
-	public int compareToReference(TypeComparator<VertexWithRankAndDangling> referencedComparator) {
-		VertexWithRankAndDanglingComparator comp = (VertexWithRankAndDanglingComparator) referencedComparator;
-		final long diff = comp.reference - this.reference;
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-	
-	@Override
-	public int compare(VertexWithRankAndDangling first, VertexWithRankAndDangling second) {
-		final long diff = first.getVertexID() - second.getVertexID();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
-		final long diff = source1.readLong() - source2.readLong();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public boolean supportsNormalizedKey() {
-		return true;
-	}
-
-	@Override
-	public int getNormalizeKeyLen() {
-		return 8;
-	}
-
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return keyBytes < 8;
-	}
-
-	@Override
-	public void putNormalizedKey(VertexWithRankAndDangling record, MemorySegment target, int offset, int len) {
-		final long value = record.getVertexID() - Long.MIN_VALUE;
-		
-		// see IntValue for an explanation of the logic
-		if (len == 8) {
-			// default case, full normalized key
-			target.putLongBigEndian(offset, value);
-		}
-		else if (len <= 0) {
-		}
-		else if (len < 8) {
-			for (int i = 0; len > 0; len--, i++) {
-				target.put(offset + i, (byte) ((value >>> ((3-i)<<3)) & 0xff));
-			}
-		}
-		else {
-			target.putLongBigEndian(offset, value);
-			for (int i = 8; i < len; i++) {
-				target.put(offset + i, (byte) 0);
-			}
-		}
-	}
-
-	@Override
-	public boolean invertNormalizedKey() {
-		return false;
-	}
-	
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return true;
-	}
-
-	@Override
-	public void writeWithKeyNormalization(VertexWithRankAndDangling record, DataOutputView target) throws IOException {
-		target.writeLong(record.getVertexID() - Long.MIN_VALUE);
-		target.writeDouble(record.getRank());
-		target.writeBoolean(record.isDangling());
-	}
-
-	@Override
-	public VertexWithRankAndDangling readWithKeyDenormalization(VertexWithRankAndDangling reuse, DataInputView source) throws IOException {
-		reuse.setVertexID(source.readLong() + Long.MIN_VALUE);
-		reuse.setRank(source.readDouble());
-		reuse.setDangling(source.readBoolean());
-		return reuse;
-	}
-
-	@Override
-	public VertexWithRankAndDanglingComparator duplicate() {
-		return new VertexWithRankAndDanglingComparator();
-	}
-
-	@Override
-	public int extractKeys(Object record, Object[] target, int index) {
-		target[index] = ((VertexWithRankAndDangling) record).getVertexID();
-		return 1;
-	}
-
-	@Override
-	@SuppressWarnings("rawtypes")
-	public TypeComparator[] getFlatComparators() {
-		return comparators;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparatorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparatorFactory.java
deleted file mode 100644
index 532ca3e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingComparatorFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.configuration.Configuration;
-
-/**
- *
- */
-public final class VertexWithRankAndDanglingComparatorFactory implements TypeComparatorFactory<VertexWithRankAndDangling> {
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithRankAndDanglingComparator createComparator() {
-		return new VertexWithRankAndDanglingComparator();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializer.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializer.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializer.java
deleted file mode 100644
index 8ff0233..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializer.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-public final class VertexWithRankAndDanglingSerializer extends TypeSerializerSingleton<VertexWithRankAndDangling> {
-
-	private static final long serialVersionUID = 1L;
-	
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public VertexWithRankAndDangling createInstance() {
-		return new VertexWithRankAndDangling();
-	}
-
-	@Override
-	public VertexWithRankAndDangling copy(VertexWithRankAndDangling from) {
-		return new VertexWithRankAndDangling(from.getVertexID(), from.getRank(), from.isDangling());
-	}
-	
-	@Override
-	public VertexWithRankAndDangling copy(VertexWithRankAndDangling from, VertexWithRankAndDangling reuse) {
-		reuse.setVertexID(from.getVertexID());
-		reuse.setRank(from.getRank());
-		reuse.setDangling(from.isDangling());
-		return reuse;
-	}
-
-	@Override
-	public int getLength() {
-		return 17;
-	}
-
-	@Override
-	public void serialize(VertexWithRankAndDangling record, DataOutputView target) throws IOException {
-		target.writeLong(record.getVertexID());
-		target.writeDouble(record.getRank());
-		target.writeBoolean(record.isDangling());
-	}
-
-	@Override
-	public VertexWithRankAndDangling deserialize(DataInputView source) throws IOException {
-		return new VertexWithRankAndDangling(source.readLong(), source.readDouble(), source.readBoolean());
-	}
-	
-	@Override
-	public VertexWithRankAndDangling deserialize(VertexWithRankAndDangling target, DataInputView source) throws IOException {
-		target.setVertexID(source.readLong());
-		target.setRank(source.readDouble());
-		target.setDangling(source.readBoolean());
-		return target;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		target.write(source, 17);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializerFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializerFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializerFactory.java
deleted file mode 100644
index 7f40d72..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDanglingSerializerFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.configuration.Configuration;
-
-
-public final class VertexWithRankAndDanglingSerializerFactory implements TypeSerializerFactory<VertexWithRankAndDangling> {
-
-	private static final VertexWithRankAndDanglingSerializer INSTANCE = new VertexWithRankAndDanglingSerializer();
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithRankAndDanglingSerializer getSerializer() {
-		return INSTANCE;
-	}
-
-	@Override
-	public Class<VertexWithRankAndDangling> getDataType() {
-		return VertexWithRankAndDangling.class;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return 1;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		return obj != null && obj.getClass() == VertexWithRankAndDanglingSerializerFactory.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
deleted file mode 100644
index 9107f4b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparator.java
+++ /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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.base.LongComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-public final class VertexWithRankComparator extends TypeComparator<VertexWithRank> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private long reference;
-
-	@SuppressWarnings("rawtypes")
-	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
-
-	@Override
-	public int hash(VertexWithRank record) {
-		final long value = record.getVertexID();
-		return 43 + (int) (value ^ value >>> 32);
-	}
-
-	@Override
-	public void setReference(VertexWithRank toCompare) {
-		this.reference = toCompare.getVertexID();
-	}
-
-	@Override
-	public boolean equalToReference(VertexWithRank candidate) {
-		return candidate.getVertexID() == this.reference;
-	}
-
-	@Override
-	public int compareToReference(TypeComparator<VertexWithRank> referencedComparator) {
-		VertexWithRankComparator comp = (VertexWithRankComparator) referencedComparator;
-		final long diff = comp.reference - this.reference;
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-	
-	@Override
-	public int compare(VertexWithRank first, VertexWithRank second) {
-		final long diff = first.getVertexID() - second.getVertexID();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
-		final long diff = source1.readLong() - source2.readLong();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public boolean supportsNormalizedKey() {
-		return true;
-	}
-
-	@Override
-	public int getNormalizeKeyLen() {
-		return 8;
-	}
-
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return keyBytes < 8;
-	}
-
-	@Override
-	public void putNormalizedKey(VertexWithRank record, MemorySegment target, int offset, int len) {
-		final long value = record.getVertexID() - Long.MIN_VALUE;
-		
-		// see IntValue for an explanation of the logic
-		if (len == 8) {
-			// default case, full normalized key
-			target.putLongBigEndian(offset, value);
-		}
-		else if (len <= 0) {
-		}
-		else if (len < 8) {
-			for (int i = 0; len > 0; len--, i++) {
-				target.put(offset + i, (byte) ((value >>> ((3-i)<<3)) & 0xff));
-			}
-		}
-		else {
-			target.putLongBigEndian(offset, value);
-			for (int i = 8; i < len; i++) {
-				target.put(offset + i, (byte) 0);
-			}
-		}
-	}
-
-	@Override
-	public boolean invertNormalizedKey() {
-		return false;
-	}
-	
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return true;
-	}
-
-	@Override
-	public void writeWithKeyNormalization(VertexWithRank record, DataOutputView target) throws IOException {
-		target.writeLong(record.getVertexID() - Long.MIN_VALUE);
-		target.writeDouble(record.getRank());
-	}
-
-	@Override
-	public VertexWithRank readWithKeyDenormalization(VertexWithRank reuse, DataInputView source) throws IOException {
-		reuse.setVertexID(source.readLong() + Long.MIN_VALUE);
-		reuse.setRank(source.readDouble());
-		return reuse;
-	}
-
-	@Override
-	public VertexWithRankComparator duplicate() {
-		return new VertexWithRankComparator();
-	}
-
-	@Override
-	public int extractKeys(Object record, Object[] target, int index) {
-		target[index] = ((VertexWithRank) record).getVertexID();
-		return 1;
-	}
-
-	@Override
-	@SuppressWarnings("rawtypes")
-	public TypeComparator[] getFlatComparators() {
-		return comparators;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparatorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparatorFactory.java
deleted file mode 100644
index 33ddb82..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankComparatorFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.configuration.Configuration;
-
-/**
- *
- */
-public final class VertexWithRankComparatorFactory implements TypeComparatorFactory<VertexWithRank> {
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithRankComparator createComparator() {
-		return new VertexWithRankComparator();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory.java
deleted file mode 100644
index 802fa16..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory.java
+++ /dev/null
@@ -1,91 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-
-
-/**
- *
- */
-public class VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory
-	implements TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithAdjacencyList>
-{
-	
-	@Override
-	public VertexWithRankDanglingToVertexWithAdjacencyListPairComparator createComparator12(
-			TypeComparator<VertexWithRankAndDangling> comparator1, TypeComparator<VertexWithAdjacencyList> comparator2)
-	{
-		return new VertexWithRankDanglingToVertexWithAdjacencyListPairComparator();
-	}
-
-	@Override
-	public VertexWithAdjacencyListToVertexWithRankDanglingPairComparator createComparator21(
-			TypeComparator<VertexWithRankAndDangling> comparator1, TypeComparator<VertexWithAdjacencyList> comparator2)
-	{
-		return new VertexWithAdjacencyListToVertexWithRankDanglingPairComparator();
-	}
-	
-
-	public static final class VertexWithRankDanglingToVertexWithAdjacencyListPairComparator
-		extends TypePairComparator<VertexWithRankAndDangling, VertexWithAdjacencyList>
-	{
-		private long reference;
-		
-		@Override
-		public void setReference(VertexWithRankAndDangling reference) {
-			this.reference = reference.getVertexID();
-		}
-		
-		@Override
-		public boolean equalToReference(VertexWithAdjacencyList candidate) {
-			return this.reference == candidate.getVertexID();
-		}
-	
-		@Override
-		public int compareToReference(VertexWithAdjacencyList candidate) {
-			long diff = candidate.getVertexID() - this.reference;
-			return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-		}
-	}
-	
-	public static final class VertexWithAdjacencyListToVertexWithRankDanglingPairComparator
-		extends TypePairComparator<VertexWithAdjacencyList, VertexWithRankAndDangling>
-	{
-		private long reference;
-		
-		@Override
-		public void setReference(VertexWithAdjacencyList reference) {
-			this.reference = reference.getVertexID();
-		}
-		
-		@Override
-		public boolean equalToReference(VertexWithRankAndDangling candidate) {
-			return this.reference == candidate.getVertexID();
-		}
-	
-		@Override
-		public int compareToReference(VertexWithRankAndDangling candidate) {
-			long diff = candidate.getVertexID() - this.reference;
-			return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithRankPairComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithRankPairComparatorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithRankPairComparatorFactory.java
deleted file mode 100644
index 674d85a..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankDanglingToVertexWithRankPairComparatorFactory.java
+++ /dev/null
@@ -1,91 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparator;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-
-
-/**
- *
- */
-public class VertexWithRankDanglingToVertexWithRankPairComparatorFactory
-	implements TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithRank>
-{
-	
-	@Override
-	public VertexWithRankDanglingToVertexWithRankComparator createComparator12(
-			TypeComparator<VertexWithRankAndDangling> comparator1, TypeComparator<VertexWithRank> comparator2)
-	{
-		return new VertexWithRankDanglingToVertexWithRankComparator();
-	}
-
-	@Override
-	public VertexWithRankToVertexWithRankDanglingPairComparator createComparator21(
-			TypeComparator<VertexWithRankAndDangling> comparator1, TypeComparator<VertexWithRank> comparator2)
-	{
-		return new VertexWithRankToVertexWithRankDanglingPairComparator();
-	}
-	
-
-	public static final class VertexWithRankDanglingToVertexWithRankComparator
-		extends TypePairComparator<VertexWithRankAndDangling, VertexWithRank>
-	{
-		private long reference;
-		
-		@Override
-		public void setReference(VertexWithRankAndDangling reference) {
-			this.reference = reference.getVertexID();
-		}
-		
-		@Override
-		public boolean equalToReference(VertexWithRank candidate) {
-			return this.reference == candidate.getVertexID();
-		}
-	
-		@Override
-		public int compareToReference(VertexWithRank candidate) {
-			long diff = candidate.getVertexID() - this.reference;
-			return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-		}
-	}
-	
-	public static final class VertexWithRankToVertexWithRankDanglingPairComparator
-		extends TypePairComparator<VertexWithRank, VertexWithRankAndDangling>
-	{
-		private long reference;
-		
-		@Override
-		public void setReference(VertexWithRank reference) {
-			this.reference = reference.getVertexID();
-		}
-		
-		@Override
-		public boolean equalToReference(VertexWithRankAndDangling candidate) {
-			return this.reference == candidate.getVertexID();
-		}
-	
-		@Override
-		public int compareToReference(VertexWithRankAndDangling candidate) {
-			long diff = candidate.getVertexID() - this.reference;
-			return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializer.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializer.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializer.java
deleted file mode 100644
index 2c3abcd..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializer.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-public final class VertexWithRankSerializer extends TypeSerializerSingleton<VertexWithRank> {
-
-	private static final long serialVersionUID = 1L;
-	
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public VertexWithRank createInstance() {
-		return new VertexWithRank();
-	}
-	
-	@Override
-	public VertexWithRank copy(VertexWithRank from) {
-		return new VertexWithRank(from.getVertexID(), from.getRank());
-	}
-
-	@Override
-	public VertexWithRank copy(VertexWithRank from, VertexWithRank reuse) {
-		reuse.setVertexID(from.getVertexID());
-		reuse.setRank(from.getRank());
-		return reuse;
-	}
-
-	@Override
-	public int getLength() {
-		return 16;
-	}
-
-	@Override
-	public void serialize(VertexWithRank record, DataOutputView target) throws IOException {
-		target.writeLong(record.getVertexID());
-		target.writeDouble(record.getRank());
-	}
-
-	@Override
-	public VertexWithRank deserialize(DataInputView source) throws IOException {
-		return new VertexWithRank(source.readLong(), source.readDouble());
-	}
-	
-	@Override
-	public VertexWithRank deserialize(VertexWithRank target, DataInputView source) throws IOException {
-		target.setVertexID(source.readLong());
-		target.setRank(source.readDouble());
-		return target;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		target.write(source, 16);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializerFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializerFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializerFactory.java
deleted file mode 100644
index 67ce028..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankSerializerFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.configuration.Configuration;
-
-
-public final class VertexWithRankSerializerFactory implements TypeSerializerFactory<VertexWithRank> {
-
-	private static final VertexWithRankSerializer INSTANCE = new VertexWithRankSerializer();
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithRankSerializer getSerializer() {
-		return INSTANCE;
-	}
-
-	@Override
-	public Class<VertexWithRank> getDataType() {
-		return VertexWithRank.class;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return 1;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		return obj != null && obj.getClass() == VertexWithRankSerializerFactory.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/AsciiLongArrayView.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/AsciiLongArrayView.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/AsciiLongArrayView.java
deleted file mode 100644
index 05b3f9b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/AsciiLongArrayView.java
+++ /dev/null
@@ -1,166 +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.test.iterative.nephele.danglingpagerank;
-
-import java.io.Serializable;
-
-import com.google.common.base.Charsets;
-
-public class AsciiLongArrayView implements Serializable {
-  private static final long serialVersionUID = 1L;
-
-  private byte[] buffer;
-
-  private int offset;
-  private int numBytes;
-
-  private int tokenOffset;
-  private int tokenNumBytes;
-
-  private static final int NOT_SET = -1;
-
-  private static final int RADIX_TEN = 10;
-  private static final long MULTMIN_RADIX_TEN = Long.MIN_VALUE / 10;
-  private static final long N_MULTMAX_RADIX_TEN = -Long.MAX_VALUE / 10;
-
-  public void set(byte[] buffer, int offset, int numBytes) {
-    this.buffer = buffer;
-    this.offset = offset;
-    this.numBytes = numBytes;
-
-    this.tokenOffset = NOT_SET;
-    checkForSingleTrailingWhitespace();
-  }
-
-  private void checkForSingleTrailingWhitespace() {
-    if (Character.isWhitespace((char) buffer[offset + numBytes -1])) {
-      numBytes--;
-    }
-  }
-
-  public int numElements() {
-    int matches = 0;
-    int pos = offset;
-    while (pos < offset + numBytes) {
-      if (Character.isWhitespace((char) buffer[pos])) {
-        matches++;
-      }
-      pos++;
-    }
-    return matches + 1;
-  }
-
-  public boolean next() {
-
-    if (tokenOffset == NOT_SET) {
-      tokenOffset = offset;
-    } else {
-      tokenOffset += tokenNumBytes + 1;
-      if (tokenOffset > offset + numBytes) {
-        return false;
-      }
-    }
-
-    tokenNumBytes = 1;
-    while (true) {
-      int candidatePos = tokenOffset + tokenNumBytes;
-      if (candidatePos >= offset + numBytes || Character.isWhitespace((char) buffer[candidatePos])) {
-        break;
-      }
-      tokenNumBytes++;
-    }
-
-    return true;
-  }
-
-  private char tokenCharAt(int pos) {
-    return (char) buffer[tokenOffset + pos];
-  }
-
-  public long element() {
-
-    long result = 0;
-    boolean negative = false;
-    int i = 0, max = tokenNumBytes;
-    long limit;
-    long multmin;
-    int digit;
-
-    if (max > 0) {
-      if (tokenCharAt(0) == '-') {
-        negative = true;
-        limit = Long.MIN_VALUE;
-        i++;
-      } else {
-        limit = -Long.MAX_VALUE;
-      }
-
-      multmin = negative ? MULTMIN_RADIX_TEN : N_MULTMAX_RADIX_TEN;
-
-      if (i < max) {
-        digit = Character.digit(tokenCharAt(i++), RADIX_TEN);
-        if (digit < 0) {
-          throw new NumberFormatException(toString());
-        } else {
-          result = -digit;
-        }
-      }
-      while (i < max) {
-        // Accumulating negatively avoids surprises near MAX_VALUE
-        digit = Character.digit(tokenCharAt(i++), RADIX_TEN);
-        if (digit < 0) {
-          throw new NumberFormatException(toString());
-        }
-        if (result < multmin) {
-          throw new NumberFormatException(toString());
-        }
-        result *= RADIX_TEN;
-        if (result < limit + digit) {
-          throw new NumberFormatException(toString());
-        }
-        result -= digit;
-      }
-    } else {
-      throw new NumberFormatException(toString());
-    }
-    if (negative) {
-      if (i > 1) {
-        return result;
-      } else { /* Only got "-" */
-        throw new NumberFormatException(toString());
-      }
-    } else {
-      return -result;
-    }
-  }
-
-//  public double elementAsDouble() {
-//    String token = new String(buffer, tokenOffset, tokenNumBytes, Charsets.US_ASCII);
-//    return Double.valueOf(token);
-//  }
-
-
-  @Override
-  public String toString() {
-    return "[" + new String(buffer, offset, numBytes, Charsets.US_ASCII) + "] (buffer length: " + buffer.length +
-        ", offset: " + offset + ", numBytes: " + numBytes + ")";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java
deleted file mode 100644
index af04557..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.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.test.iterative.nephele.danglingpagerank;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-
-public class BooleanValue implements Value {
-  private static final long serialVersionUID = 1L;
-
-  private boolean value;
-
-  public BooleanValue(boolean value) {
-    this.value = value;
-  }
-
-  public BooleanValue() {
-  }
-
-  public boolean get() {
-    return value;
-  }
-
-  public void set(boolean value) {
-    this.value = value;
-  }
-
-  @Override
-  public void write(DataOutputView out) throws IOException {
-    out.writeBoolean(value);
-  }
-
-  @Override
-  public void read(DataInputView in) throws IOException {
-    value = in.readBoolean();
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
deleted file mode 100644
index 78038b3..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ /dev/null
@@ -1,295 +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.test.iterative.nephele.danglingpagerank;
-
-import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.common.typeutils.record.RecordComparatorFactory;
-import org.apache.flink.api.common.typeutils.record.RecordPairComparatorFactory;
-import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver;
-import org.apache.flink.runtime.operators.CoGroupDriver;
-import org.apache.flink.runtime.operators.CollectorMapDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.test.iterative.nephele.JobGraphUtils;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.util.OperatingSystem;
-
-public class CompensatableDanglingPageRank {
-	
-	private static final TypeSerializerFactory<?> recSerializer = RecordSerializerFactory.get();
-	
-	@SuppressWarnings("unchecked")
-	private static final TypeComparatorFactory<?> fieldZeroComparator = new RecordComparatorFactory(new int[] {0}, new Class[] {LongValue.class}, new boolean[] {true});
-	
-	private static final TypePairComparatorFactory<?, ?> pairComparatorFactory = new RecordPairComparatorFactory();
-	
-	
-	private static final int NUM_FILE_HANDLES_PER_SORT = 64;
-	
-	private static final float SORT_SPILL_THRESHOLD = 0.85f;
-	
-	private static final int ITERATION_ID = 1;
-	
-
-//	public static void main(String[] args) throws Exception {
-//		String confPath = args.length >= 6 ? confPath = args[5] : PlayConstants.PLAY_DIR + "local-conf";
-//		
-//		GlobalConfiguration.loadConfiguration(confPath);
-//		Configuration conf = GlobalConfiguration.getConfiguration();
-//		
-//		JobGraph jobGraph = getJobGraph(args);
-//		JobGraphUtils.submit(jobGraph, conf);
-//	}
-		
-	public static JobGraph getJobGraph(String[] args) throws Exception {
-
-		int parallelism = 2;
-		String pageWithRankInputPath = ""; // "file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
-		String adjacencyListInputPath = ""; // "file://" + PlayConstants.PLAY_DIR +
-//			"test-inputs/danglingpagerank/adjacencylists";
-		String outputPath = OperatingSystem.isWindows() ? "file:/c:/tmp/flink/iterations" : "file:///tmp/flink/iterations";
-//		String confPath = PlayConstants.PLAY_DIR + "local-conf";
-		int minorConsumer = 25;
-		int matchMemory = 50;
-		int coGroupSortMemory = 50;
-		int numIterations = 25;
-		long numVertices = 5;
-		long numDanglingVertices = 1;
-
-		String failingWorkers = "1";
-		int failingIteration = 2;
-		double messageLoss = 0.75;
-
-		if (args.length >= 15) {
-			parallelism = Integer.parseInt(args[0]);
-			pageWithRankInputPath = args[1];
-			adjacencyListInputPath = args[2];
-			outputPath = args[3];
-//			confPath = args[4];
-			minorConsumer = Integer.parseInt(args[5]);
-			matchMemory = Integer.parseInt(args[6]);
-			coGroupSortMemory = Integer.parseInt(args[7]);
-			numIterations = Integer.parseInt(args[8]);
-			numVertices = Long.parseLong(args[9]);
-			numDanglingVertices = Long.parseLong(args[10]);
-			failingWorkers = args[11];
-			failingIteration = Integer.parseInt(args[12]);
-			messageLoss = Double.parseDouble(args[13]);
-		}
-
-		int totalMemoryConsumption = 3*minorConsumer + matchMemory + coGroupSortMemory;
-
-		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
-		
-		// --------------- the inputs ---------------------
-
-		// page rank input
-		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
-		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
-		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		pageWithRankInputConfig.setOutputComparator(fieldZeroComparator, 0);
-		pageWithRankInputConfig.setOutputSerializer(recSerializer);
-		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-
-		// edges as adjacency list
-		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
-		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
-		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		adjacencyListInputConfig.setOutputSerializer(recSerializer);
-		adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0);
-
-		// --------------- the head ---------------------
-		JobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, parallelism);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		headConfig.setIterationId(ITERATION_ID);
-		
-		// initial input / partial solution
-		headConfig.addInputToGroup(0);
-		headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
-		headConfig.setInputSerializer(recSerializer, 0);
-		headConfig.setInputComparator(fieldZeroComparator, 0);
-		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setRelativeMemoryInput(0, (double)minorConsumer/totalMemoryConsumption);
-		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
-		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
-		
-		// back channel / iterations
-		headConfig.setRelativeBackChannelMemory((double)minorConsumer/totalMemoryConsumption);
-		
-		// output into iteration
-		headConfig.setOutputSerializer(recSerializer);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		
-		// final output
-		TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
-		headFinalOutConfig.setOutputSerializer(recSerializer);
-		headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);
-		
-		// the sync
-		headConfig.setIterationHeadIndexOfSyncOutput(3);
-		headConfig.setNumberOfIterations(numIterations);
-		
-		// the driver 
-		headConfig.setDriver(CollectorMapDriver.class);
-		headConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-		headConfig.setStubWrapper(new UserCodeClassWrapper<CompensatingMap>(CompensatingMap.class));
-		headConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		headConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		headConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		headConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		headConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-
-		// --------------- the join ---------------------
-		
-		JobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, parallelism);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-		intermediateConfig.setIterationId(ITERATION_ID);
-//		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
-		intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class);
-		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption);
-		intermediateConfig.addInputToGroup(0);
-		intermediateConfig.addInputToGroup(1);
-		intermediateConfig.setInputSerializer(recSerializer, 0);
-		intermediateConfig.setInputSerializer(recSerializer, 1);
-		intermediateConfig.setDriverComparator(fieldZeroComparator, 0);
-		intermediateConfig.setDriverComparator(fieldZeroComparator, 1);
-		intermediateConfig.setDriverPairComparator(pairComparatorFactory);
-		
-		intermediateConfig.setOutputSerializer(recSerializer);
-		intermediateConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		intermediateConfig.setOutputComparator(fieldZeroComparator, 0);
-		
-		intermediateConfig.setStubWrapper(new UserCodeClassWrapper<CompensatableDotProductMatch>(CompensatableDotProductMatch.class));
-		intermediateConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		intermediateConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		intermediateConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		intermediateConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-
-		// ---------------- the tail (co group) --------------------
-		
-		JobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			parallelism);
-		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
-		tailConfig.setIterationId(ITERATION_ID);
-		tailConfig.setIsWorksetUpdate();
-		// TODO we need to combine!
-		
-		// inputs and driver
-		tailConfig.setDriver(CoGroupDriver.class);
-		tailConfig.setDriverStrategy(DriverStrategy.CO_GROUP);
-		tailConfig.addInputToGroup(0);
-		tailConfig.addInputToGroup(1);
-		tailConfig.setInputSerializer(recSerializer, 0);
-		tailConfig.setInputSerializer(recSerializer, 1);
-		tailConfig.setDriverComparator(fieldZeroComparator, 0);
-		tailConfig.setDriverComparator(fieldZeroComparator, 1);
-		tailConfig.setDriverPairComparator(pairComparatorFactory);
-		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
-		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
-		tailConfig.setInputComparator(fieldZeroComparator, 1);
-		tailConfig.setRelativeMemoryInput(1, (double)coGroupSortMemory/totalMemoryConsumption);
-		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
-		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
-		
-		// output
-		tailConfig.setOutputSerializer(recSerializer);
-		
-		// the stub
-		tailConfig.setStubWrapper(new UserCodeClassWrapper<CompensatableDotProductCoGroup>(CompensatableDotProductCoGroup.class));
-		tailConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		tailConfig.setStubParameter("pageRank.numDanglingVertices", String.valueOf(numDanglingVertices));
-		tailConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		tailConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		tailConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		
-		// --------------- the output ---------------------
-
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
-		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
-		outputConfig.addInputToGroup(0);
-		outputConfig.setInputSerializer(recSerializer, 0);
-		outputConfig.setStubWrapper(new UserCodeClassWrapper<PageWithRankOutFormat>(PageWithRankOutFormat.class));
-		outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath);
-		
-		// --------------- the auxiliaries ---------------------
-
-		JobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setNumberOfIterations(numIterations);
-		syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-		syncConfig.setConvergenceCriterion(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new DiffL1NormConvergenceCriterion());
-		syncConfig.setIterationId(ITERATION_ID);
-		
-		// --------------- the wiring ---------------------
-
-		JobGraphUtils.connect(pageWithRankInput, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.POINTWISE);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		
-		JobGraphUtils.connect(adjacencyListInput, intermediate, DistributionPattern.ALL_TO_ALL);
-		
-		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-		
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		pageWithRankInput.setSlotSharingGroup(sharingGroup);
-		adjacencyListInput.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		tail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-		
-		tail.setStrictlyCoLocatedWith(head);
-		intermediate.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java
deleted file mode 100644
index bd1eeb9..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductCoGroup.java
+++ /dev/null
@@ -1,137 +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.test.iterative.nephele.danglingpagerank;
-
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.flink.api.java.record.functions.CoGroupFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("deprecation")
-public class CompensatableDotProductCoGroup extends CoGroupFunction {
-	
-	private static final long serialVersionUID = 1L;
-	
-
-	public static final String AGGREGATOR_NAME = "pagerank.aggregator";
-
-	private Record accumulator = new Record();
-
-	private int workerIndex;
-
-	private int currentIteration;
-
-	private int failingIteration;
-
-	private Set<Integer> failingWorkers;
-
-	private PageRankStatsAggregator aggregator;
-
-	private long numVertices;
-
-	private long numDanglingVertices;
-
-	private double dampingFactor;
-
-	private double danglingRankFactor;
-
-	private static final double BETA = 0.85;
-
-	private final DoubleValue newRank = new DoubleValue();
-
-	private BooleanValue isDangling = new BooleanValue();
-
-	private LongValue vertexID = new LongValue();
-
-	private DoubleValue doubleInstance = new DoubleValue();
-
-	@Override
-	public void open(Configuration parameters) {
-		workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		
-		failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-		numDanglingVertices = ConfigUtils.asLong("pageRank.numDanglingVertices", parameters);
-
-		dampingFactor = (1d - BETA) / (double) numVertices;
-		
-		aggregator = getIterationRuntimeContext().getIterationAggregator(AGGREGATOR_NAME);
-		
-		if (currentIteration == 1) {
-			danglingRankFactor = BETA * (double) numDanglingVertices / ((double) numVertices * (double) numVertices);
-		} else {
-			PageRankStats previousAggregate = getIterationRuntimeContext().getPreviousIterationAggregate(AGGREGATOR_NAME);
-			danglingRankFactor = BETA * previousAggregate.danglingRank() / (double) numVertices;
-		}
-	}
-
-	@Override
-	public void coGroup(Iterator<Record> currentPageRankIterator, Iterator<Record> partialRanks, Collector<Record> collector) {
-
-		if (!currentPageRankIterator.hasNext()) {
-			long missingVertex = partialRanks.next().getField(0, LongValue.class).getValue();
-			throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!");
-		}
-
-		Record currentPageRank = currentPageRankIterator.next();
-
-		long edges = 0;
-		double summedRank = 0;
-		while (partialRanks.hasNext()) {
-			Record pr = partialRanks.next();
-			summedRank += pr.getField(1, doubleInstance).getValue();
-			edges++;
-		}
-
-		double rank = BETA * summedRank + dampingFactor + danglingRankFactor;
-
-		double currentRank = currentPageRank.getField(1, doubleInstance).getValue();
-		isDangling = currentPageRank.getField(2, isDangling);
-
-		double danglingRankToAggregate = isDangling.get() ? rank : 0;
-		long danglingVerticesToAggregate = isDangling.get() ? 1 : 0;
-
-		double diff = Math.abs(currentRank - rank);
-
-		aggregator.aggregate(diff, rank, danglingRankToAggregate, danglingVerticesToAggregate, 1, edges, summedRank, 0);
-
-		newRank.setValue(rank);
-
-		accumulator.setField(0, currentPageRank.getField(0, vertexID));
-		accumulator.setField(1, newRank);
-		accumulator.setField(2, isDangling);
-
-		collector.collect(accumulator);
-	}
-
-	@Override
-	public void close() throws Exception {
-		if (currentIteration == failingIteration && failingWorkers.contains(workerIndex)) {
-			aggregator.reset();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductMatch.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductMatch.java
deleted file mode 100644
index 0508886..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDotProductMatch.java
+++ /dev/null
@@ -1,102 +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.test.iterative.nephele.danglingpagerank;
-
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.flink.api.java.record.functions.JoinFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("deprecation")
-public class CompensatableDotProductMatch extends JoinFunction {
-
-	private static final long serialVersionUID = 1L;
-
-	private Record record;
-
-	private LongValue vertexID;
-
-	private DoubleValue partialRank;
-
-	private DoubleValue rank = new DoubleValue();
-
-	private LongArrayView adjacentNeighbors = new LongArrayView();
-
-	private int workerIndex;
-
-	private int currentIteration;
-
-	private int failingIteration;
-
-	private Set<Integer> failingWorkers;
-
-	private double messageLoss;
-
-	private Random random;
-
-	@Override
-	public void open(Configuration parameters) {
-		record = new Record();
-		vertexID = new LongValue();
-		partialRank = new DoubleValue();
-
-		workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		messageLoss = ConfigUtils.asDouble("compensation.messageLoss", parameters);
-
-		random = new Random();
-	}
-
-	@Override
-	public void join(Record pageWithRank, Record adjacencyList, Collector<Record> collector) {
-
-		rank = pageWithRank.getField(1, rank);
-		adjacentNeighbors = adjacencyList.getField(1, adjacentNeighbors);
-		int numNeighbors = adjacentNeighbors.size();
-
-		double rankToDistribute = rank.getValue() / (double) numNeighbors;
-
-		partialRank.setValue(rankToDistribute);
-		record.setField(1, partialRank);
-
-		boolean isFailure = currentIteration == failingIteration && failingWorkers.contains(workerIndex);
-
-		for (int n = 0; n < numNeighbors; n++) {
-			vertexID.setValue(adjacentNeighbors.getQuick(n));
-			record.setField(0, vertexID);
-
-			if (isFailure) {
-				if (random.nextDouble() >= messageLoss) {
-					collector.collect(record);
-				}
-			} else {
-				collector.collect(record);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatingMap.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatingMap.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatingMap.java
deleted file mode 100644
index d8189ef..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatingMap.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.test.iterative.nephele.danglingpagerank;
-
-import java.util.Set;
-
-import org.apache.flink.api.java.record.functions.MapFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("deprecation")
-public class CompensatingMap extends MapFunction {
-
-	private static final long serialVersionUID = 1L;
-
-	private int workerIndex;
-
-	private int currentIteration;
-
-	private long numVertices;
-
-	private int failingIteration;
-
-	private Set<Integer> failingWorkers;
-
-	private double uniformRank;
-
-	private double rescaleFactor;
-
-	private DoubleValue rank = new DoubleValue();
-
-	@Override
-	public void open(Configuration parameters) {
-
-		workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-
-		if (currentIteration > 1) {
-			PageRankStats stats = (PageRankStats) getIterationRuntimeContext().getPreviousIterationAggregate(
-				CompensatableDotProductCoGroup.AGGREGATOR_NAME);
-
-			uniformRank = 1d / (double) numVertices;
-			double lostMassFactor = (numVertices - stats.numVertices()) / (double) numVertices;
-			rescaleFactor = (1 - lostMassFactor) / stats.rank();
-		}
-	}
-
-	@Override
-	public void map(Record pageWithRank, Collector<Record> out) {
-
-		if (currentIteration == failingIteration + 1) {
-
-			rank = pageWithRank.getField(1, rank);
-
-			if (failingWorkers.contains(workerIndex)) {
-				rank.setValue(uniformRank);
-			} else {
-				rank.setValue(rank.getValue() * rescaleFactor);
-			}
-			pageWithRank.setField(1, rank);
-		}
-		
-		out.collect(pageWithRank);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DanglingPageGenerateRankInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DanglingPageGenerateRankInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DanglingPageGenerateRankInputFormat.java
deleted file mode 100644
index 4da2bf2..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DanglingPageGenerateRankInputFormat.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.test.iterative.nephele.danglingpagerank;
-
-import java.util.regex.Pattern;
-
-import org.apache.flink.api.java.record.io.TextInputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class DanglingPageGenerateRankInputFormat extends TextInputFormat {
-
-	private static final long serialVersionUID = 1L;
-
-	private DoubleValue initialRank;
-
-	private static final Pattern SEPARATOR = Pattern.compile("[, \t]");
-
-	@Override
-	public void configure(Configuration parameters) {
-		long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-		initialRank = new DoubleValue(1 / (double) numVertices);
-		super.configure(parameters);
-	}
-
-	@Override
-	public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) {
-		String str = new String(bytes, offset, numBytes);
-
-		String[] tokens = SEPARATOR.split(str);
-
-		long vertexID = Long.parseLong(tokens[0]);
-		boolean isDangling = tokens.length > 1 && Integer.parseInt(tokens[1]) == 1;
-
-		target.clear();
-		target.addField(new LongValue(vertexID));
-		target.addField(initialRank);
-		target.addField(new BooleanValue(isDangling));
-
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DiffL1NormConvergenceCriterion.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DiffL1NormConvergenceCriterion.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DiffL1NormConvergenceCriterion.java
deleted file mode 100644
index 9868173..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/DiffL1NormConvergenceCriterion.java
+++ /dev/null
@@ -1,44 +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.test.iterative.nephele.danglingpagerank;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
-
-@SuppressWarnings("serial")
-public class DiffL1NormConvergenceCriterion implements ConvergenceCriterion<PageRankStats> {
-
-	private static final double EPSILON = 0.00005;
-
-	private static final Logger log = LoggerFactory.getLogger(DiffL1NormConvergenceCriterion.class);
-
-	@Override
-	public boolean isConverged(int iteration, PageRankStats pageRankStats) {
-		double diff = pageRankStats.diff();
-
-		if (log.isInfoEnabled()) {
-			log.info("Stats in iteration [" + iteration + "]: " + pageRankStats);
-			log.info("L1 norm of the vector difference is [" + diff + "] in iteration [" + iteration + "]");
-		}
-
-		return diff < EPSILON;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedAdjacencyListInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedAdjacencyListInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedAdjacencyListInputFormat.java
deleted file mode 100644
index df2fed6..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedAdjacencyListInputFormat.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.test.iterative.nephele.danglingpagerank;
-
-import org.apache.flink.api.java.record.io.DelimitedInputFormat;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class ImprovedAdjacencyListInputFormat extends DelimitedInputFormat {
-  private static final long serialVersionUID = 1L;
-
-  private final LongValue vertexID = new LongValue();
-  private final AsciiLongArrayView arrayView = new AsciiLongArrayView();
-  private final LongArrayView adjacentVertices = new LongArrayView();
-
-  @Override
-  public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) {
-
-    if (numBytes == 0) {
-      return null;
-    }
-
-    arrayView.set(bytes, offset, numBytes);
-
-    int numElements = arrayView.numElements();
-    adjacentVertices.allocate(numElements - 1);
-
-    try {
-
-      int pos = 0;
-      while (arrayView.next()) {
-
-        if (pos == 0) {
-          vertexID.setValue(arrayView.element());
-        } else {
-          adjacentVertices.setQuick(pos - 1, arrayView.element());
-        }
-
-        pos++;
-      }
-
-      //sanity check
-      if (pos != numElements) {
-        throw new IllegalStateException("Should have gotten " + numElements + " elements, but saw " + pos);
-      }
-
-    } catch (RuntimeException e) {
-      throw new RuntimeException("Error parsing: " + arrayView.toString(), e);
-    }
-
-    target.clear();
-    target.addField(vertexID);
-    target.addField(adjacentVertices);
-    return target;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedDanglingPageRankInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedDanglingPageRankInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedDanglingPageRankInputFormat.java
deleted file mode 100644
index cfa64c8..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/ImprovedDanglingPageRankInputFormat.java
+++ /dev/null
@@ -1,73 +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.test.iterative.nephele.danglingpagerank;
-
-import org.apache.flink.api.java.record.io.DelimitedInputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-
-public class ImprovedDanglingPageRankInputFormat extends DelimitedInputFormat {
-	private static final long serialVersionUID = 1L;
-
-	private LongValue vertexID = new LongValue();
-
-	private DoubleValue initialRank;
-
-	private BooleanValue isDangling = new BooleanValue();
-
-	private AsciiLongArrayView arrayView = new AsciiLongArrayView();
-
-	private static final long DANGLING_MARKER = 1l;
-
-	@Override
-	public void configure(Configuration parameters) {
-		long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-		initialRank = new DoubleValue(1 / (double) numVertices);
-		super.configure(parameters);
-	}
-
-	@Override
-	public Record readRecord(Record target, byte[] bytes, int offset, int numBytes) {
-		arrayView.set(bytes, offset, numBytes);
-
-		try {
-			arrayView.next();
-			vertexID.setValue(arrayView.element());
-
-			if (arrayView.next()) {
-				isDangling.set(arrayView.element() == DANGLING_MARKER);
-			} else {
-				isDangling.set(false);
-			}
-
-		} catch (NumberFormatException e) {
-			throw new RuntimeException("Error parsing " + arrayView.toString(), e);
-		}
-
-		target.clear();
-		target.addField(vertexID);
-		target.addField(initialRank);
-		target.addField(isDangling);
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/LongArrayView.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/LongArrayView.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/LongArrayView.java
deleted file mode 100644
index 825b210..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/LongArrayView.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.test.iterative.nephele.danglingpagerank;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-
-public class LongArrayView implements Value {
-  private static final long serialVersionUID = 1L;
-
-  private long[] entries = new long[0];
-  private int numEntries = 0;
-
-  public LongArrayView() {
-  }
-
-  public long get(int index) {
-    if (index >= numEntries) {
-      throw new ArrayIndexOutOfBoundsException();
-    }
-    return getQuick(index);
-  }
-
-  public long getQuick(int index) {
-    return entries[index];
-  }
-
-  public void allocate(int numEntries) {
-    this.numEntries = numEntries;
-    ensureCapacity();
-  }
-
-  public void set(int index, long value) {
-    if (index >= numEntries) {
-      throw new ArrayIndexOutOfBoundsException();
-    }
-    setQuick(index, value);
-  }
-
-  public void setQuick(int index, long value) {
-    entries[index] = value;
-  }
-
-  public int size() {
-    return numEntries;
-  }
-
-  private void ensureCapacity() {
-    if (entries.length < numEntries) {
-      entries = new long[numEntries];
-    }
-  }
-
-  public void write(DataOutputView out) throws IOException {
-    out.writeInt(numEntries);
-    for (int n = 0; n < numEntries; n++) {
-      out.writeLong(entries[n]);
-    }
-  }
-
-  public void read(DataInputView in) throws IOException {
-    numEntries = in.readInt();
-    ensureCapacity();
-    for (int n = 0; n < numEntries; n++) {
-      entries[n] = in.readLong();
-    }
-  }
-}


[4/5] flink git commit: [FLINK-1681] [tests] Remove outdated 'nephele' iteration tests.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
deleted file mode 100644
index a6e6b6e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ /dev/null
@@ -1,315 +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.test.iterative.nephele.customdanglingpagerank;
-
-import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver;
-import org.apache.flink.runtime.operators.CoGroupDriver;
-import org.apache.flink.runtime.operators.CollectorMapDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.test.iterative.nephele.JobGraphUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyListComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyListSerializerFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDanglingComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDanglingSerializerFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankDanglingToVertexWithRankPairComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankSerializerFactory;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.DiffL1NormConvergenceCriterion;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator;
-import org.apache.flink.test.util.TestBaseUtils;
-
-public class CustomCompensatableDanglingPageRank {
-	
-	private static final int NUM_FILE_HANDLES_PER_SORT = 64;
-	
-	private static final float SORT_SPILL_THRESHOLD = 0.85f;
-	
-	private static final int ITERATION_ID = 1;
-	
-	
-	private static TypeSerializerFactory<VertexWithRank> vertexWithRankSerializer = new VertexWithRankSerializerFactory();
-	
-	private static TypeSerializerFactory<VertexWithRankAndDangling> vertexWithRankAndDanglingSerializer = new VertexWithRankAndDanglingSerializerFactory();
-	
-	private static TypeSerializerFactory<VertexWithAdjacencyList> vertexWithAdjacencyListSerializer = new VertexWithAdjacencyListSerializerFactory();
-	
-	private static TypeComparatorFactory<VertexWithRank> vertexWithRankComparator = new VertexWithRankComparatorFactory();
-	
-	private static TypeComparatorFactory<VertexWithRankAndDangling> vertexWithRankAndDanglingComparator = new VertexWithRankAndDanglingComparatorFactory();
-	
-	private static TypeComparatorFactory<VertexWithAdjacencyList> vertexWithAdjacencyListComparator = new VertexWithAdjacencyListComparatorFactory();
-	
-	private static TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithAdjacencyList> matchComparator =
-			new VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory();
-	
-	private static TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithRank> coGroupComparator =
-			new VertexWithRankDanglingToVertexWithRankPairComparatorFactory();
-	
-
-//	public static void main(String[] args) throws Exception {
-//		String confPath = args.length >= 6 ? confPath = args[5] : PlayConstants.PLAY_DIR + "local-conf";
-//		
-//		GlobalConfiguration.loadConfiguration(confPath);
-//		Configuration conf = GlobalConfiguration.getConfiguration();
-//		
-//		JobGraph jobGraph = getJobGraph(args);
-//		JobGraphUtils.submit(jobGraph, conf);
-//	}
-		
-	public static JobGraph getJobGraph(String[] args) throws Exception {
-
-		int parallelism = 2;
-		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
-		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
-//			"test-inputs/danglingpagerank/adjacencylists";
-		String outputPath = TestBaseUtils.constructTestURI(CustomCompensatableDanglingPageRank.class, "flink_iterations");
-//		String confPath = PlayConstants.PLAY_DIR + "local-conf";
-		int minorConsumer = 2;
-		int matchMemory = 5;
-		int coGroupSortMemory = 5;
-		int numIterations = 25;
-		long numVertices = 5;
-		long numDanglingVertices = 1;
-
-		String failingWorkers = "1";
-		int failingIteration = 2;
-		double messageLoss = 0.75;
-
-		if (args.length >= 14) {
-			parallelism = Integer.parseInt(args[0]);
-			pageWithRankInputPath = args[1];
-			adjacencyListInputPath = args[2];
-			outputPath = args[3];
-//			confPath = args[4];
-			minorConsumer = Integer.parseInt(args[5]);
-			matchMemory = Integer.parseInt(args[6]);
-			coGroupSortMemory = Integer.parseInt(args[7]);
-			numIterations = Integer.parseInt(args[8]);
-			numVertices = Long.parseLong(args[9]);
-			numDanglingVertices = Long.parseLong(args[10]);
-			failingWorkers = args[11];
-			failingIteration = Integer.parseInt(args[12]);
-			messageLoss = Double.parseDouble(args[13]);
-		}
-
-		int totalMemoryConsumption = 3*minorConsumer + matchMemory + coGroupSortMemory;
-
-		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
-		
-		// --------------- the inputs ---------------------
-
-		// page rank input
-		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
-		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
-		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
-		pageWithRankInputConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-
-		// edges as adjacency list
-		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
-		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
-		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
-		adjacencyListInputConfig.setOutputComparator(vertexWithAdjacencyListComparator, 0);
-
-		// --------------- the head ---------------------
-		JobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			parallelism);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		headConfig.setIterationId(ITERATION_ID);
-		
-		// initial input / partial solution
-		headConfig.addInputToGroup(0);
-		headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
-		headConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		headConfig.setInputComparator(vertexWithRankAndDanglingComparator, 0);
-		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setRelativeMemoryInput(0, (double) minorConsumer / totalMemoryConsumption);
-		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
-		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
-		
-		// back channel / iterations
-		headConfig.setRelativeBackChannelMemory((double) minorConsumer / totalMemoryConsumption);
-		
-		// output into iteration
-		headConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		
-		// final output
-		TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
-		headFinalOutConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);
-		
-		// the sync
-		headConfig.setIterationHeadIndexOfSyncOutput(3);
-		headConfig.setNumberOfIterations(numIterations);
-		
-		// the driver 
-		headConfig.setDriver(CollectorMapDriver.class);
-		headConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-		headConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatingMap>(CustomCompensatingMap.class));
-		headConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		headConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		headConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		headConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		headConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-
-		// --------------- the join ---------------------
-		
-		JobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, parallelism);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-		intermediateConfig.setIterationId(ITERATION_ID);
-//		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
-		intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class);
-		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setRelativeMemoryDriver((double) matchMemory / totalMemoryConsumption);
-		intermediateConfig.addInputToGroup(0);
-		intermediateConfig.addInputToGroup(1);
-		intermediateConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		intermediateConfig.setInputSerializer(vertexWithAdjacencyListSerializer, 1);
-		intermediateConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
-		intermediateConfig.setDriverComparator(vertexWithAdjacencyListComparator, 1);
-		intermediateConfig.setDriverPairComparator(matchComparator);
-		
-		intermediateConfig.setOutputSerializer(vertexWithRankSerializer);
-		intermediateConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		intermediateConfig.setOutputComparator(vertexWithRankComparator, 0);
-		
-		intermediateConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductMatch>(CustomCompensatableDotProductMatch.class));
-		intermediateConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		intermediateConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		intermediateConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		intermediateConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-
-		// ---------------- the tail (co group) --------------------
-		
-		JobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			parallelism);
-		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
-		tailConfig.setIterationId(ITERATION_ID);
-		tailConfig.setIsWorksetUpdate();
-		
-		// inputs and driver
-		tailConfig.setDriver(CoGroupDriver.class);
-		tailConfig.setDriverStrategy(DriverStrategy.CO_GROUP);
-		tailConfig.addInputToGroup(0);
-		tailConfig.addInputToGroup(1);
-		tailConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		tailConfig.setInputSerializer(vertexWithRankSerializer, 1);
-		tailConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
-		tailConfig.setDriverComparator(vertexWithRankComparator, 1);
-		tailConfig.setDriverPairComparator(coGroupComparator);
-		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
-		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
-		tailConfig.setInputComparator(vertexWithRankComparator, 1);
-		tailConfig.setRelativeMemoryInput(1, (double) coGroupSortMemory / totalMemoryConsumption);
-		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
-		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
-		
-		// output
-		tailConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		
-		// the stub
-		tailConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductCoGroup>(CustomCompensatableDotProductCoGroup.class));
-		tailConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		tailConfig.setStubParameter("pageRank.numDanglingVertices", String.valueOf(numDanglingVertices));
-		tailConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		tailConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		tailConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		
-		// --------------- the output ---------------------
-
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
-		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
-		outputConfig.addInputToGroup(0);
-		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		outputConfig.setStubWrapper(new UserCodeClassWrapper<CustomPageWithRankOutFormat>(CustomPageWithRankOutFormat.class));
-		outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath);
-		
-		// --------------- the auxiliaries ---------------------
-
-		JobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setNumberOfIterations(numIterations);
-		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-		syncConfig.setConvergenceCriterion(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new DiffL1NormConvergenceCriterion());
-		syncConfig.setIterationId(ITERATION_ID);
-		
-		// --------------- the wiring ---------------------
-
-		JobGraphUtils.connect(pageWithRankInput, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.POINTWISE);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		
-		JobGraphUtils.connect(adjacencyListInput, intermediate, DistributionPattern.ALL_TO_ALL);
-		
-		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-		
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		pageWithRankInput.setSlotSharingGroup(sharingGroup);
-		adjacencyListInput.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		tail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-		
-		tail.setStrictlyCoLocatedWith(head);
-		intermediate.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
deleted file mode 100644
index 0bf258f..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ /dev/null
@@ -1,329 +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.test.iterative.nephele.customdanglingpagerank;
-
-import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.api.java.record.io.FileOutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
-import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
-import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.runtime.operators.BuildSecondCachedJoinDriver;
-import org.apache.flink.runtime.operators.CoGroupDriver;
-import org.apache.flink.runtime.operators.CollectorMapDriver;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.chaining.SynchronousChainedCombineDriver;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.test.iterative.nephele.JobGraphUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyListComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyListSerializerFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDanglingComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDanglingSerializerFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankDanglingToVertexWithRankPairComparatorFactory;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankSerializerFactory;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.DiffL1NormConvergenceCriterion;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator;
-import org.apache.flink.test.util.TestBaseUtils;
-
-public class CustomCompensatableDanglingPageRankWithCombiner {
-	
-	private static final int NUM_FILE_HANDLES_PER_SORT = 64;
-	
-	private static final float SORT_SPILL_THRESHOLD = 0.85f;
-	
-	private static final int ITERATION_ID = 1;
-	
-	
-	private static TypeSerializerFactory<VertexWithRank> vertexWithRankSerializer = new VertexWithRankSerializerFactory();
-	
-	private static TypeSerializerFactory<VertexWithRankAndDangling> vertexWithRankAndDanglingSerializer = new VertexWithRankAndDanglingSerializerFactory();
-	
-	private static TypeSerializerFactory<VertexWithAdjacencyList> vertexWithAdjacencyListSerializer = new VertexWithAdjacencyListSerializerFactory();
-	
-	private static TypeComparatorFactory<VertexWithRank> vertexWithRankComparator = new VertexWithRankComparatorFactory();
-	
-	private static TypeComparatorFactory<VertexWithRankAndDangling> vertexWithRankAndDanglingComparator = new VertexWithRankAndDanglingComparatorFactory();
-	
-	private static TypeComparatorFactory<VertexWithAdjacencyList> vertexWithAdjacencyListComparator = new VertexWithAdjacencyListComparatorFactory();
-	
-	private static TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithAdjacencyList> matchComparator =
-			new VertexWithRankDanglingToVertexWithAdjacencyListPairComparatorFactory();
-	
-	private static TypePairComparatorFactory<VertexWithRankAndDangling, VertexWithRank> coGroupComparator =
-			new VertexWithRankDanglingToVertexWithRankPairComparatorFactory();
-	
-
-//	public static void main(String[] args) throws Exception {
-//		String confPath = args.length >= 6 ? confPath = args[5] : PlayConstants.PLAY_DIR + "local-conf";
-//		
-//		GlobalConfiguration.loadConfiguration(confPath);
-//		Configuration conf = GlobalConfiguration.getConfiguration();
-//		
-//		JobGraph jobGraph = getJobGraph(args);
-//		JobGraphUtils.submit(jobGraph, conf);
-//	}
-		
-	public static JobGraph getJobGraph(String[] args) throws Exception {
-
-		int parallelism = 2;
-		String pageWithRankInputPath = ""; //"file://" + PlayConstants.PLAY_DIR + "test-inputs/danglingpagerank/pageWithRank";
-		String adjacencyListInputPath = ""; //"file://" + PlayConstants.PLAY_DIR +
-//			"test-inputs/danglingpagerank/adjacencylists";
-		String outputPath =  TestBaseUtils.constructTestURI(CustomCompensatableDanglingPageRankWithCombiner.class, "flink_iterations");
-		int minorConsumer = 2;
-		int matchMemory = 5;
-		int coGroupSortMemory = 5;
-		int numIterations = 25;
-		long numVertices = 5;
-		long numDanglingVertices = 1;
-
-		String failingWorkers = "1";
-		int failingIteration = 2;
-		double messageLoss = 0.75;
-
-		if (args.length >= 14) {
-			parallelism = Integer.parseInt(args[0]);
-			pageWithRankInputPath = args[1];
-			adjacencyListInputPath = args[2];
-			outputPath = args[3];
-			// [4] is config path
-			minorConsumer = Integer.parseInt(args[5]);
-			matchMemory = Integer.parseInt(args[6]);
-			coGroupSortMemory = Integer.parseInt(args[7]);
-			numIterations = Integer.parseInt(args[8]);
-			numVertices = Long.parseLong(args[9]);
-			numDanglingVertices = Long.parseLong(args[10]);
-			failingWorkers = args[11];
-			failingIteration = Integer.parseInt(args[12]);
-			messageLoss = Double.parseDouble(args[13]);
-		}
-
-		int totalMemoryConsumption = 3*minorConsumer + 2*coGroupSortMemory + matchMemory;
-
-		JobGraph jobGraph = new JobGraph("CompensatableDanglingPageRank");
-		
-		// --------------- the inputs ---------------------
-
-		// page rank input
-		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
-			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, parallelism);
-		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
-		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		pageWithRankInputConfig.setOutputComparator(vertexWithRankAndDanglingComparator, 0);
-		pageWithRankInputConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-
-		// edges as adjacency list
-		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
-			adjacencyListInputPath, "AdjancencyListInput", jobGraph, parallelism);
-		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
-		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		adjacencyListInputConfig.setOutputSerializer(vertexWithAdjacencyListSerializer);
-		adjacencyListInputConfig.setOutputComparator(vertexWithAdjacencyListComparator, 0);
-
-		// --------------- the head ---------------------
-		JobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			parallelism);
-		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
-		headConfig.setIterationId(ITERATION_ID);
-		
-		// initial input / partial solution
-		headConfig.addInputToGroup(0);
-		headConfig.setIterationHeadPartialSolutionOrWorksetInputIndex(0);
-		headConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		headConfig.setInputComparator(vertexWithRankAndDanglingComparator, 0);
-		headConfig.setInputLocalStrategy(0, LocalStrategy.SORT);
-		headConfig.setRelativeMemoryInput(0, (double)minorConsumer/totalMemoryConsumption);
-		headConfig.setFilehandlesInput(0, NUM_FILE_HANDLES_PER_SORT);
-		headConfig.setSpillingThresholdInput(0, SORT_SPILL_THRESHOLD);
-		
-		// back channel / iterations
-		headConfig.setRelativeBackChannelMemory((double)minorConsumer/totalMemoryConsumption);
-		
-		// output into iteration
-		headConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		
-		// final output
-		TaskConfig headFinalOutConfig = new TaskConfig(new Configuration());
-		headFinalOutConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		headFinalOutConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		headConfig.setIterationHeadFinalOutputConfig(headFinalOutConfig);
-		
-		// the sync
-		headConfig.setIterationHeadIndexOfSyncOutput(3);
-		headConfig.setNumberOfIterations(numIterations);
-		
-		// the driver 
-		headConfig.setDriver(CollectorMapDriver.class);
-		headConfig.setDriverStrategy(DriverStrategy.COLLECTOR_MAP);
-		headConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatingMap>(CustomCompensatingMap.class));
-		headConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		headConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		headConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		headConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		headConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-
-		// --------------- the join ---------------------
-		
-		JobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, parallelism);
-		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
-		intermediateConfig.setIterationId(ITERATION_ID);
-//		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
-		intermediateConfig.setDriver(BuildSecondCachedJoinDriver.class);
-		intermediateConfig.setDriverStrategy(DriverStrategy.HYBRIDHASH_BUILD_SECOND);
-		intermediateConfig.setRelativeMemoryDriver((double)matchMemory/totalMemoryConsumption);
-		intermediateConfig.addInputToGroup(0);
-		intermediateConfig.addInputToGroup(1);
-		intermediateConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		intermediateConfig.setInputSerializer(vertexWithAdjacencyListSerializer, 1);
-		intermediateConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
-		intermediateConfig.setDriverComparator(vertexWithAdjacencyListComparator, 1);
-		intermediateConfig.setDriverPairComparator(matchComparator);
-		
-		intermediateConfig.setOutputSerializer(vertexWithRankSerializer);
-		intermediateConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
-		
-		intermediateConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductMatch>(CustomCompensatableDotProductMatch.class));
-		intermediateConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		intermediateConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		intermediateConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		intermediateConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		
-		// the combiner and the output
-		TaskConfig combinerConfig = new TaskConfig(new Configuration());
-		combinerConfig.addInputToGroup(0);
-		combinerConfig.setInputSerializer(vertexWithRankSerializer, 0);
-		combinerConfig.setDriverStrategy(DriverStrategy.SORTED_GROUP_COMBINE);
-		combinerConfig.setDriverComparator(vertexWithRankComparator, 0);
-		combinerConfig.setDriverComparator(vertexWithRankComparator, 1);
-		combinerConfig.setRelativeMemoryDriver((double)coGroupSortMemory/totalMemoryConsumption);
-		combinerConfig.setOutputSerializer(vertexWithRankSerializer);
-		combinerConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
-		combinerConfig.setOutputComparator(vertexWithRankComparator, 0);
-		combinerConfig.setStubWrapper(new UserCodeClassWrapper<CustomRankCombiner>(CustomRankCombiner.class));
-		intermediateConfig.addChainedTask(SynchronousChainedCombineDriver.class, combinerConfig, "Combiner");
-
-		// ---------------- the tail (co group) --------------------
-		
-		JobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			parallelism);
-		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
-		tailConfig.setIterationId(ITERATION_ID);
-		tailConfig.setIsWorksetUpdate();
-		
-		// inputs and driver
-		tailConfig.setDriver(CoGroupDriver.class);
-		tailConfig.setDriverStrategy(DriverStrategy.CO_GROUP);
-		tailConfig.addInputToGroup(0);
-		tailConfig.addInputToGroup(1);
-		tailConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		tailConfig.setInputSerializer(vertexWithRankSerializer, 1);
-		tailConfig.setDriverComparator(vertexWithRankAndDanglingComparator, 0);
-		tailConfig.setDriverComparator(vertexWithRankComparator, 1);
-		tailConfig.setDriverPairComparator(coGroupComparator);
-		tailConfig.setInputAsynchronouslyMaterialized(0, true);
-		tailConfig.setRelativeInputMaterializationMemory(0, (double)minorConsumer/totalMemoryConsumption);
-		tailConfig.setInputLocalStrategy(1, LocalStrategy.SORT);
-		tailConfig.setInputComparator(vertexWithRankComparator, 1);
-		tailConfig.setRelativeMemoryInput(1, (double)coGroupSortMemory/totalMemoryConsumption);
-		tailConfig.setFilehandlesInput(1, NUM_FILE_HANDLES_PER_SORT);
-		tailConfig.setSpillingThresholdInput(1, SORT_SPILL_THRESHOLD);
-		tailConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-		
-		// output
-		tailConfig.setOutputSerializer(vertexWithRankAndDanglingSerializer);
-		
-		// the stub
-		tailConfig.setStubWrapper(new UserCodeClassWrapper<CustomCompensatableDotProductCoGroup>(CustomCompensatableDotProductCoGroup.class));
-		tailConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
-		tailConfig.setStubParameter("pageRank.numDanglingVertices", String.valueOf(numDanglingVertices));
-		tailConfig.setStubParameter("compensation.failingWorker", failingWorkers);
-		tailConfig.setStubParameter("compensation.failingIteration", String.valueOf(failingIteration));
-		tailConfig.setStubParameter("compensation.messageLoss", String.valueOf(messageLoss));
-		
-		// --------------- the output ---------------------
-
-		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", parallelism);
-		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
-		outputConfig.addInputToGroup(0);
-		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
-		outputConfig.setStubWrapper(new UserCodeClassWrapper<CustomPageWithRankOutFormat>(CustomPageWithRankOutFormat.class));
-		outputConfig.setStubParameter(FileOutputFormat.FILE_PARAMETER_KEY, outputPath);
-		
-		// --------------- the auxiliaries ---------------------
-
-		JobVertex sync = JobGraphUtils.createSync(jobGraph, parallelism);
-		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setNumberOfIterations(numIterations);
-		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
-		syncConfig.setConvergenceCriterion(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new DiffL1NormConvergenceCriterion());
-		syncConfig.setIterationId(ITERATION_ID);
-		
-		// --------------- the wiring ---------------------
-
-		JobGraphUtils.connect(pageWithRankInput, head, DistributionPattern.ALL_TO_ALL);
-
-		JobGraphUtils.connect(head, intermediate, DistributionPattern.POINTWISE);
-		intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		
-		JobGraphUtils.connect(adjacencyListInput, intermediate, DistributionPattern.ALL_TO_ALL);
-		
-		JobGraphUtils.connect(head, tail, DistributionPattern.POINTWISE);
-		JobGraphUtils.connect(intermediate, tail, DistributionPattern.ALL_TO_ALL);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
-		tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, parallelism);
-
-		JobGraphUtils.connect(head, output, DistributionPattern.POINTWISE);
-
-		JobGraphUtils.connect(head, sync, DistributionPattern.POINTWISE);
-		
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		pageWithRankInput.setSlotSharingGroup(sharingGroup);
-		adjacencyListInput.setSlotSharingGroup(sharingGroup);
-		head.setSlotSharingGroup(sharingGroup);
-		intermediate.setSlotSharingGroup(sharingGroup);
-		tail.setSlotSharingGroup(sharingGroup);
-		output.setSlotSharingGroup(sharingGroup);
-		sync.setSlotSharingGroup(sharingGroup);
-
-		tail.setStrictlyCoLocatedWith(head);
-		intermediate.setStrictlyCoLocatedWith(head);
-
-		return jobGraph;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java
deleted file mode 100644
index faf777e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductCoGroup.java
+++ /dev/null
@@ -1,130 +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.test.iterative.nephele.customdanglingpagerank;
-
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStats;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStatsAggregator;
-import org.apache.flink.util.Collector;
-
-public class CustomCompensatableDotProductCoGroup extends AbstractRichFunction implements CoGroupFunction<VertexWithRankAndDangling, VertexWithRank, VertexWithRankAndDangling> {
-
-	private static final long serialVersionUID = 1L;
-
-	public static final String AGGREGATOR_NAME = "pagerank.aggregator";
-	
-	private VertexWithRankAndDangling accumulator = new VertexWithRankAndDangling();
-
-	private PageRankStatsAggregator aggregator;
-
-	private long numVertices;
-
-	private long numDanglingVertices;
-
-	private double dampingFactor;
-
-	private double danglingRankFactor;
-
-	private static final double BETA = 0.85;
-	
-	private int workerIndex;
-	
-	private int currentIteration;
-	
-	private int failingIteration;
-	
-	private Set<Integer> failingWorkers;
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		
-		failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-		numDanglingVertices = ConfigUtils.asLong("pageRank.numDanglingVertices", parameters);
-		
-		dampingFactor = (1d - BETA) / (double) numVertices;
-
-		aggregator = getIterationRuntimeContext().getIterationAggregator(AGGREGATOR_NAME);
-		
-		if (currentIteration == 1) {
-			danglingRankFactor = BETA * (double) numDanglingVertices / ((double) numVertices * (double) numVertices);
-		} else {
-			PageRankStats previousAggregate = getIterationRuntimeContext().getPreviousIterationAggregate(AGGREGATOR_NAME);
-			danglingRankFactor = BETA * previousAggregate.danglingRank() / (double) numVertices;
-		}
-	}
-
-	@Override
-	public void coGroup(Iterable<VertexWithRankAndDangling> currentPageRankIterable, Iterable<VertexWithRank> partialRanks,
-			Collector<VertexWithRankAndDangling> collector)
-	{
-		final Iterator<VertexWithRankAndDangling> currentPageRankIterator = currentPageRankIterable.iterator();
-		
-		if (!currentPageRankIterator.hasNext()) {
-			long missingVertex = partialRanks.iterator().next().getVertexID();
-			throw new IllegalStateException("No current page rank for vertex [" + missingVertex + "]!");
-		}
-
-		VertexWithRankAndDangling currentPageRank = currentPageRankIterator.next();
-
-		long edges = 0;
-		double summedRank = 0;
-		for (VertexWithRank pr :partialRanks) {
-			summedRank += pr.getRank();
-			edges++;
-		}
-
-		double rank = BETA * summedRank + dampingFactor + danglingRankFactor;
-
-		double currentRank = currentPageRank.getRank();
-		boolean isDangling = currentPageRank.isDangling();
-
-		double danglingRankToAggregate = isDangling ? rank : 0;
-		long danglingVerticesToAggregate = isDangling ? 1 : 0;
-
-		double diff = Math.abs(currentRank - rank);
-
-		aggregator.aggregate(diff, rank, danglingRankToAggregate, danglingVerticesToAggregate, 1, edges, summedRank, 0);
-
-		accumulator.setVertexID(currentPageRank.getVertexID());
-		accumulator.setRank(rank);
-		accumulator.setDangling(isDangling);
-
-		collector.collect(accumulator);
-	}
-
-	@Override
-	public void close() throws Exception {
-		if (currentIteration == failingIteration && failingWorkers.contains(workerIndex)) {
-			aggregator.reset();
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java
deleted file mode 100644
index cc20aba..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDotProductMatch.java
+++ /dev/null
@@ -1,80 +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.test.iterative.nephele.customdanglingpagerank;
-
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.util.Collector;
-
-public class CustomCompensatableDotProductMatch extends AbstractRichFunction implements
-		FlatJoinFunction<VertexWithRankAndDangling, VertexWithAdjacencyList, VertexWithRank>
-{
-	private static final long serialVersionUID = 1L;
-	
-	
-	private VertexWithRank record = new VertexWithRank();
-
-	private Random random = new Random();
-	
-	private double messageLoss;
-	
-	private boolean isFailure;
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		int workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		int currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		int failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		Set<Integer> failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		isFailure = currentIteration == failingIteration && failingWorkers.contains(workerIndex);
-		messageLoss = ConfigUtils.asDouble("compensation.messageLoss", parameters);
-	}
-
-	@Override
-	public void join(VertexWithRankAndDangling pageWithRank, VertexWithAdjacencyList adjacencyList, Collector<VertexWithRank> collector)
-	throws Exception
-	{
-		double rank = pageWithRank.getRank();
-		long[] adjacentNeighbors = adjacencyList.getTargets();
-		int numNeighbors = adjacencyList.getNumTargets();
-
-		double rankToDistribute = rank / (double) numNeighbors;
-		record.setRank(rankToDistribute);
-
-		for (int n = 0; n < numNeighbors; n++) {
-			record.setVertexID(adjacentNeighbors[n]);
-			if (isFailure) {
-				if (random.nextDouble() >= messageLoss) {
-					collector.collect(record);
-				}
-			} else {
-				collector.collect(record);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.java
deleted file mode 100644
index d1bf03c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatingMap.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.test.iterative.nephele.customdanglingpagerank;
-
-import java.util.Set;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.GenericCollectorMap;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.PageRankStats;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("deprecation")
-public class CustomCompensatingMap extends AbstractRichFunction implements GenericCollectorMap<VertexWithRankAndDangling, VertexWithRankAndDangling> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	
-	private boolean isFailureIteration;
-
-	private boolean isFailingWorker;
-
-	private double uniformRank;
-
-	private double rescaleFactor;
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		int currentIteration = getIterationRuntimeContext().getSuperstepNumber();
-		int failingIteration = ConfigUtils.asInteger("compensation.failingIteration", parameters);
-		isFailureIteration = currentIteration == failingIteration + 1;
-		
-		int workerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		Set<Integer> failingWorkers = ConfigUtils.asIntSet("compensation.failingWorker", parameters);
-		isFailingWorker = failingWorkers.contains(workerIndex);
-		
-		long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-
-		if (currentIteration > 1) {
-			
-			PageRankStats stats = (PageRankStats) getIterationRuntimeContext().getPreviousIterationAggregate(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME);
-
-			uniformRank = 1d / (double) numVertices;
-			double lostMassFactor = (numVertices - stats.numVertices()) / (double) numVertices;
-			rescaleFactor = (1 - lostMassFactor) / stats.rank();
-		}
-	}
-
-	@Override
-	public void map(VertexWithRankAndDangling pageWithRank, Collector<VertexWithRankAndDangling> out) throws Exception {
-
-		if (isFailureIteration) {
-			double rank = pageWithRank.getRank();
-
-			if (isFailingWorker) {
-				pageWithRank.setRank(uniformRank);
-			} else {
-				pageWithRank.setRank(rank * rescaleFactor);
-			}
-		}
-		out.collect(pageWithRank);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedAdjacencyListInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedAdjacencyListInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedAdjacencyListInputFormat.java
deleted file mode 100644
index 863c081..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedAdjacencyListInputFormat.java
+++ /dev/null
@@ -1,66 +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.test.iterative.nephele.customdanglingpagerank;
-
-import org.apache.flink.api.common.io.DelimitedInputFormat;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithAdjacencyList;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.AsciiLongArrayView;
-
-public class CustomImprovedAdjacencyListInputFormat extends DelimitedInputFormat<VertexWithAdjacencyList> {
-	private static final long serialVersionUID = 1L;
-
-	private final AsciiLongArrayView arrayView = new AsciiLongArrayView();
-
-	@Override
-	public VertexWithAdjacencyList readRecord(VertexWithAdjacencyList target, byte[] bytes, int offset, int numBytes) {
-
-		if (numBytes == 0) {
-			return null;
-		}
-
-		arrayView.set(bytes, offset, numBytes);
-		
-		long[] list = target.getTargets();
-
-		try {
-
-			int pos = 0;
-			while (arrayView.next()) {
-
-				if (pos == 0) {
-					target.setVertexID(arrayView.element());
-				} else {
-					if (list.length <= pos - 1) {
-						list = new long[list.length < 16 ? 16 : list.length * 2];
-						target.setTargets(list);
-					}
-					list[pos - 1] = arrayView.element();
-				}
-				pos++;
-			}
-			
-			target.setNumTargets(pos - 1);
-		} catch (RuntimeException e) {
-			throw new RuntimeException("Error parsing: " + arrayView.toString(), e);
-		}
-
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedDanglingPageRankInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedDanglingPageRankInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedDanglingPageRankInputFormat.java
deleted file mode 100644
index ec5b6d1..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomImprovedDanglingPageRankInputFormat.java
+++ /dev/null
@@ -1,66 +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.test.iterative.nephele.customdanglingpagerank;
-
-import org.apache.flink.api.common.io.DelimitedInputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.iterative.nephele.ConfigUtils;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-import org.apache.flink.test.iterative.nephele.danglingpagerank.AsciiLongArrayView;
-
-public class CustomImprovedDanglingPageRankInputFormat extends DelimitedInputFormat<VertexWithRankAndDangling> {
-	private static final long serialVersionUID = 1L;
-
-	private AsciiLongArrayView arrayView = new AsciiLongArrayView();
-
-	private static final long DANGLING_MARKER = 1l;
-	
-	private double initialRank;
-
-	@Override
-	public void configure(Configuration parameters) {
-		long numVertices = ConfigUtils.asLong("pageRank.numVertices", parameters);
-		initialRank = 1.0 / numVertices;
-		super.configure(parameters);
-	}
-
-	@Override
-	public VertexWithRankAndDangling readRecord(VertexWithRankAndDangling target, byte[] bytes, int offset, int numBytes) {
-
-		arrayView.set(bytes, offset, numBytes);
-
-		try {
-			arrayView.next();
-			target.setVertexID(arrayView.element());
-
-			if (arrayView.next()) {
-				target.setDangling(arrayView.element() == DANGLING_MARKER);
-			} else {
-				target.setDangling(false);
-			}
-
-		} catch (NumberFormatException e) {
-			throw new RuntimeException("Error parsing " + arrayView.toString(), e);
-		}
-
-		target.setRank(initialRank);
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomPageWithRankOutFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomPageWithRankOutFormat.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomPageWithRankOutFormat.java
deleted file mode 100644
index b4ffb25..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomPageWithRankOutFormat.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.test.iterative.nephele.customdanglingpagerank;
-
-import com.google.common.base.Charsets;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.io.FileOutputFormat;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRankAndDangling;
-
-public class CustomPageWithRankOutFormat extends FileOutputFormat<VertexWithRankAndDangling> {
-	private static final long serialVersionUID = 1L;
-
-	private final StringBuilder buffer = new StringBuilder();
-
-	@Override
-	public void writeRecord(VertexWithRankAndDangling record) throws IOException {
-		buffer.setLength(0);
-		buffer.append(record.getVertexID());
-		buffer.append('\t');
-		buffer.append(record.getRank());
-		buffer.append('\n');
-
-		byte[] bytes = buffer.toString().getBytes(Charsets.UTF_8);
-		stream.write(bytes);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.java
deleted file mode 100644
index e2a160d..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomRankCombiner.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.test.iterative.nephele.customdanglingpagerank;
-
-import java.util.Iterator;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.test.iterative.nephele.customdanglingpagerank.types.VertexWithRank;
-import org.apache.flink.util.Collector;
-
-
-public class CustomRankCombiner extends AbstractRichFunction implements GroupReduceFunction<VertexWithRank, VertexWithRank>,
-		GroupCombineFunction<VertexWithRank, VertexWithRank>
-{
-	private static final long serialVersionUID = 1L;
-	
-	private final VertexWithRank accumulator = new VertexWithRank();
-	
-	@Override
-	public void reduce(Iterable<VertexWithRank> records, Collector<VertexWithRank> out) throws Exception {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public void combine(Iterable<VertexWithRank> recordsIterable, Collector<VertexWithRank> out) throws Exception {
-		final Iterator<VertexWithRank> records = recordsIterable.iterator();
-		
-		VertexWithRank next = records.next();
-		this.accumulator.setVertexID(next.getVertexID());
-		double rank = next.getRank();
-		
-		while (records.hasNext()) {
-			rank += records.next().getRank();
-		}
-		
-		this.accumulator.setRank(rank);
-		out.collect(this.accumulator);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyList.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyList.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyList.java
deleted file mode 100644
index d19dd59..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyList.java
+++ /dev/null
@@ -1,83 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-
-/**
- *
- */
-public final class VertexWithAdjacencyList {
-	
-	private static final long[] EMPTY = new long[0];
-	
-	private long vertexID;
-	
-	private long[] targets;
-	
-	private int numTargets;
-	
-	public VertexWithAdjacencyList() {
-		this.targets = EMPTY;
-	}
-	
-	public VertexWithAdjacencyList(long vertexID, long[] targets) {
-		this.vertexID = vertexID;
-		this.targets = targets;
-	}
-
-	
-	public long getVertexID() {
-		return vertexID;
-	}
-	
-	public void setVertexID(long vertexID) {
-		this.vertexID = vertexID;
-	}
-	
-	public long[] getTargets() {
-		return targets;
-	}
-	
-	public void setTargets(long[] targets) {
-		this.targets = targets;
-	}
-	
-	public int getNumTargets() {
-		return numTargets;
-	}
-	
-	public void setNumTargets(int numTargets) {
-		this.numTargets = numTargets;
-	}
-	
-
-	@Override
-	public String toString() {
-		StringBuilder bld = new StringBuilder(32);
-		bld.append(this.vertexID);
-		bld.append(" : ");
-		for (int i = 0; i < this.numTargets; i++) {
-			if (i != 0) {
-				bld.append(',');
-			}
-			bld.append(this.targets[i]);
-		}
-		return bld.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
deleted file mode 100644
index 7d11530..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparator.java
+++ /dev/null
@@ -1,148 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.base.LongComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-public final class VertexWithAdjacencyListComparator extends TypeComparator<VertexWithAdjacencyList> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private long reference;
-
-	@SuppressWarnings("rawtypes")
-	private TypeComparator[] comparators = new TypeComparator[]{new LongComparator(true)};
-
-	@Override
-	public int hash(VertexWithAdjacencyList record) {
-		final long value = record.getVertexID();
-		return 43 + (int) (value ^ value >>> 32);
-	}
-
-	@Override
-	public void setReference(VertexWithAdjacencyList toCompare) {
-		this.reference = toCompare.getVertexID();
-	}
-
-	@Override
-	public boolean equalToReference(VertexWithAdjacencyList candidate) {
-		return candidate.getVertexID() == this.reference;
-	}
-
-	@Override
-	public int compareToReference(TypeComparator<VertexWithAdjacencyList> referencedComparator) {
-		VertexWithAdjacencyListComparator comp = (VertexWithAdjacencyListComparator) referencedComparator;
-		final long diff = comp.reference - this.reference;
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-	
-	@Override
-	public int compare(VertexWithAdjacencyList first, VertexWithAdjacencyList second) {
-		final long diff = first.getVertexID() - second.getVertexID();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public int compareSerialized(DataInputView source1, DataInputView source2) throws IOException {
-		final long diff = source1.readLong() - source2.readLong();
-		return diff < 0 ? -1 : diff > 0 ? 1 : 0;
-	}
-
-	@Override
-	public boolean supportsNormalizedKey() {
-		return true;
-	}
-
-	@Override
-	public int getNormalizeKeyLen() {
-		return 8;
-	}
-
-	@Override
-	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
-		return keyBytes < 8;
-	}
-
-	@Override
-	public void putNormalizedKey(VertexWithAdjacencyList record, MemorySegment target, int offset, int len) {
-		final long value = record.getVertexID() - Long.MIN_VALUE;
-		
-		// see IntValue for an explanation of the logic
-		if (len == 8) {
-			// default case, full normalized key
-			target.putLongBigEndian(offset, value);
-		}
-		else if (len <= 0) {
-		}
-		else if (len < 8) {
-			for (int i = 0; len > 0; len--, i++) {
-				target.put(offset + i, (byte) ((value >>> ((3-i)<<3)) & 0xff));
-			}
-		}
-		else {
-			target.putLongBigEndian(offset, value);
-			for (int i = 8; i < len; i++) {
-				target.put(offset + i, (byte) 0);
-			}
-		}
-	}
-
-	@Override
-	public boolean invertNormalizedKey() {
-		return false;
-	}
-	
-	@Override
-	public boolean supportsSerializationWithKeyNormalization() {
-		return false;
-	}
-
-	@Override
-	public void writeWithKeyNormalization(VertexWithAdjacencyList record, DataOutputView target) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public VertexWithAdjacencyList readWithKeyDenormalization(VertexWithAdjacencyList reuse, DataInputView source) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public VertexWithAdjacencyListComparator duplicate() {
-		return new VertexWithAdjacencyListComparator();
-	}
-
-	@Override
-	public int extractKeys(Object record, Object[] target, int index) {
-		target[index] = ((VertexWithAdjacencyList) record).getVertexID();
-		return 1;
-	}
-	
-	@Override
-	@SuppressWarnings("rawtypes")
-	public TypeComparator[] getFlatComparators() {
-		return comparators;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparatorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparatorFactory.java
deleted file mode 100644
index 0bc3263..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListComparatorFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
-import org.apache.flink.configuration.Configuration;
-
-/**
- *
- */
-public final class VertexWithAdjacencyListComparatorFactory implements TypeComparatorFactory<VertexWithAdjacencyList> {
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithAdjacencyListComparator createComparator() {
-		return new VertexWithAdjacencyListComparator();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializer.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializer.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializer.java
deleted file mode 100644
index 751ced3..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializer.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-public final class VertexWithAdjacencyListSerializer extends TypeSerializerSingleton<VertexWithAdjacencyList> {
-
-	private static final long serialVersionUID = 1L;
-	
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public VertexWithAdjacencyList createInstance() {
-		return new VertexWithAdjacencyList();
-	}
-
-	@Override
-	public VertexWithAdjacencyList copy(VertexWithAdjacencyList from) {
-		VertexWithAdjacencyList copy = new VertexWithAdjacencyList(from.getVertexID(), new long[from.getNumTargets()]);
-		copy.setNumTargets(from.getNumTargets());
-		System.arraycopy(from.getTargets(), 0, copy.getTargets(), 0, from.getNumTargets());
-		return copy;
-	}
-	
-	@Override
-	public VertexWithAdjacencyList copy(VertexWithAdjacencyList from, VertexWithAdjacencyList reuse) {
-		if (reuse.getTargets().length < from.getTargets().length) {
-			reuse.setTargets(new long[from.getTargets().length]);
-		}
-		
-		reuse.setVertexID(from.getVertexID());
-		reuse.setNumTargets(from.getNumTargets());
-		System.arraycopy(from.getTargets(), 0, reuse.getTargets(), 0, from.getNumTargets());
-		return reuse;
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public void serialize(VertexWithAdjacencyList record, DataOutputView target) throws IOException {
-		target.writeLong(record.getVertexID());
-		
-		final long[] targets = record.getTargets();
-		final int numTargets = record.getNumTargets();
-		target.writeInt(numTargets);
-		
-		for (int i = 0; i < numTargets; i++) {
-			target.writeLong(targets[i]);
-		}
-	}
-
-	@Override
-	public VertexWithAdjacencyList deserialize(DataInputView source) throws IOException {
-		return deserialize(new VertexWithAdjacencyList(), source);
-	}
-	
-	@Override
-	public VertexWithAdjacencyList deserialize(VertexWithAdjacencyList target, DataInputView source) throws IOException {
-		target.setVertexID(source.readLong());
-		
-		final int numTargets = source.readInt();
-		long[] targets = target.getTargets();
-		if (targets.length < numTargets) {
-			targets = new long[numTargets];
-			target.setTargets(targets);
-		}
-		
-		target.setNumTargets(numTargets);
-		
-		for (int i = 0; i < numTargets; i++) {
-			targets[i] = source.readLong();
-		}
-		return target;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		target.write(source, 8);
-		
-		final int numTargets = source.readInt();
-		target.writeInt(numTargets);
-		target.write(source, numTargets * 8);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializerFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializerFactory.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializerFactory.java
deleted file mode 100644
index cdd50a6..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithAdjacencyListSerializerFactory.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.test.iterative.nephele.customdanglingpagerank.types;
-
-import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
-import org.apache.flink.configuration.Configuration;
-
-
-public final class VertexWithAdjacencyListSerializerFactory implements TypeSerializerFactory<VertexWithAdjacencyList> {
-
-	private static final VertexWithAdjacencyListSerializer INSTANCE = new VertexWithAdjacencyListSerializer();
-	
-	@Override
-	public void writeParametersToConfig(Configuration config) {}
-
-	@Override
-	public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException {}
-
-	@Override
-	public VertexWithAdjacencyListSerializer getSerializer() {
-		return INSTANCE;
-	}
-
-	@Override
-	public Class<VertexWithAdjacencyList> getDataType() {
-		return VertexWithAdjacencyList.class;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return 1;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		return obj != null && obj.getClass() == VertexWithAdjacencyListSerializerFactory.class;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRank.java
deleted file mode 100644
index e62185e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRank.java
+++ /dev/null
@@ -1,65 +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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.Serializable;
-
-
-/**
- *
- */
-public final class VertexWithRank implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private long vertexID;
-	
-	private double rank;
-
-	
-	public VertexWithRank() {
-	}
-	
-	public VertexWithRank(long vertexID, double rank) {
-		this.vertexID = vertexID;
-		this.rank = rank;
-	}
-
-	
-	public long getVertexID() {
-		return vertexID;
-	}
-	
-	public void setVertexID(long vertexID) {
-		this.vertexID = vertexID;
-	}
-	
-	public double getRank() {
-		return rank;
-	}
-	
-	public void setRank(double rank) {
-		this.rank = rank;
-	}
-	
-
-	@Override
-	public String toString() {
-		return this.vertexID + " - " + this.rank;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/a7a57ebe/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDangling.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDangling.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDangling.java
deleted file mode 100644
index 4cdb594..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/types/VertexWithRankAndDangling.java
+++ /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.test.iterative.nephele.customdanglingpagerank.types;
-
-import java.io.Serializable;
-
-
-/**
- *
- */
-public final class VertexWithRankAndDangling implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private long vertexID;
-	
-	private double rank;
-
-	private boolean dangling;
-	
-
-	public VertexWithRankAndDangling() {
-	}
-	
-	public VertexWithRankAndDangling(long vertexID, double rank, boolean dangling) {
-		this.vertexID = vertexID;
-		this.rank = rank;
-		this.dangling = dangling;
-	}
-
-	
-	public long getVertexID() {
-		return vertexID;
-	}
-	
-	public void setVertexID(long vertexID) {
-		this.vertexID = vertexID;
-	}
-	
-	public double getRank() {
-		return rank;
-	}
-	
-	public void setRank(double rank) {
-		this.rank = rank;
-	}
-	
-	public boolean isDangling() {
-		return dangling;
-	}
-	
-	public void setDangling(boolean dangling) {
-		this.dangling = dangling;
-	}
-	
-
-	@Override
-	public String toString() {
-		return this.vertexID + " - " + this.rank + (this.isDangling() ? " (dangling)" : "");
-	}
-}