You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gr...@apache.org on 2017/07/12 23:44:01 UTC

[01/22] flink git commit: [FLINK-7132] [java] Fix BulkIteration parallelism

Repository: flink
Updated Branches:
  refs/heads/master 1da93a4c7 -> 9bd491e05


[FLINK-7132] [java] Fix BulkIteration parallelism

Copy the parallelism when translating a BulkIteration.

This closes #4283


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

Branch: refs/heads/master
Commit: 480ccfb4809871d1a0812442a8e175d82aa186c1
Parents: 1da93a4
Author: Greg Hogan <co...@greghogan.com>
Authored: Fri Jul 7 09:17:33 2017 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Wed Jul 12 18:37:37 2017 -0400

----------------------------------------------------------------------
 .../api/java/operators/OperatorTranslation.java |  8 +-
 .../BulkIterationTranslationTest.java           | 93 ++++++++++++++++++++
 2 files changed, 99 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/480ccfb4/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index cdbec71..22b9186 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -209,10 +209,14 @@ public class OperatorTranslation {
 	private <T> BulkIterationBase<T> translateBulkIteration(BulkIterationResultSet<?> untypedIterationEnd) {
 		@SuppressWarnings("unchecked")
 		BulkIterationResultSet<T> iterationEnd = (BulkIterationResultSet<T>) untypedIterationEnd;
-		
+		IterativeDataSet<T> iterationHead = iterationEnd.getIterationHead();
+
 		BulkIterationBase<T> iterationOperator =
 				new BulkIterationBase<>(new UnaryOperatorInformation<>(iterationEnd.getType(), iterationEnd.getType()), "Bulk Iteration");
-		IterativeDataSet<T> iterationHead = iterationEnd.getIterationHead();
+
+		if (iterationHead.getParallelism() > 0) {
+			iterationOperator.setParallelism(iterationHead.getParallelism());
+		}
 
 		translated.put(iterationHead, iterationOperator.getPartialSolution());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/480ccfb4/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java
new file mode 100644
index 0000000..11a2dab
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/BulkIterationTranslationTest.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.api.java.operators.translation;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.operators.base.BulkIterationBase;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.api.java.tuple.Tuple3;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Simple test for validating the parallelism of a bulk iteration. This test is
+ * not as comprehensive as {@link DeltaIterationTranslationTest}.
+ */
+@SuppressWarnings("serial")
+public class BulkIterationTranslationTest implements java.io.Serializable {
+
+	@Test
+	public void testCorrectTranslation() {
+		final String jobName = "Test JobName";
+
+		final int numIterations = 13;
+
+		final int defaultParallelism = 133;
+		final int iterationParallelism = 77;
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// ------------ construct the test program ------------------
+
+		{
+			env.setParallelism(defaultParallelism);
+
+			@SuppressWarnings("unchecked")
+			DataSet<Tuple3<Double, Long, String>> initialDataSet = env.fromElements(new Tuple3<>(3.44, 5L, "abc"));
+
+			IterativeDataSet<Tuple3<Double, Long, String>> bulkIteration = initialDataSet.iterate(numIterations);
+			bulkIteration.setParallelism(iterationParallelism);
+
+			// test that multiple iteration consumers are supported
+			DataSet<Tuple3<Double, Long, String>> identity = bulkIteration
+				.map(new IdentityMapper<Tuple3<Double, Long, String>>());
+
+			DataSet<Tuple3<Double, Long, String>> result = bulkIteration.closeWith(identity);
+
+			result.output(new DiscardingOutputFormat<Tuple3<Double, Long, String>>());
+			result.writeAsText("/dev/null");
+		}
+
+		Plan p = env.createProgramPlan(jobName);
+
+		// ------------- validate the plan ----------------
+
+		BulkIterationBase<?> iteration = (BulkIterationBase<?>) p.getDataSinks().iterator().next().getInput();
+
+		assertEquals(jobName, p.getJobName());
+		assertEquals(defaultParallelism, p.getDefaultParallelism());
+		assertEquals(iterationParallelism, iteration.getParallelism());
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private static class IdentityMapper<T> extends RichMapFunction<T, T> {
+		@Override
+		public T map(T value) throws Exception {
+			return value;
+		}
+	}
+}


[13/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
deleted file mode 100644
index fb3e589..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapITCase.java
+++ /dev/null
@@ -1,514 +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.javaApiOperators;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Assert;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class MapITCase extends MultipleProgramsTestBase {
-
-	public MapITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testIdentityMapWithBasicType() throws Exception {
-		/*
-		 * Test identity map with basic type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> identityMapDs = ds.
-				map(new Mapper1());
-
-		List<String> result = identityMapDs.collect();
-
-		String expected = "Hi\n" +
-				"Hello\n" +
-				"Hello world\n" +
-				"Hello world, how are you?\n" +
-				"I am fine.\n" +
-				"Luke Skywalker\n" +
-				"Random comment\n" +
-				"LOL\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRuntimeContextAndExecutionConfigParams() throws Exception {
-		/*
-		 * Test identity map with basic type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().setNumberOfExecutionRetries(1000);
-		env.getConfig().setTaskCancellationInterval(50000);
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> identityMapDs = ds.
-			map(new RichMapFunction<String, String>() {
-				@Override
-				public String map(String value) throws Exception {
-					Assert.assertTrue(1000 == getRuntimeContext().getExecutionConfig().getNumberOfExecutionRetries());
-					Assert.assertTrue(50000 == getRuntimeContext().getExecutionConfig().getTaskCancellationInterval());
-					return value;
-				}
-			});
-
-		List<String> result = identityMapDs.collect();
-
-		String expected = "Hi\n" +
-			"Hello\n" +
-			"Hello world\n" +
-			"Hello world, how are you?\n" +
-			"I am fine.\n" +
-			"Luke Skywalker\n" +
-			"Random comment\n" +
-			"LOL\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper1 implements MapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map(String value) throws Exception {
-			return value;
-		}
-	}
-
-	@Test
-	public void testIdentityMapWithTuple() throws Exception {
-		/*
-		 * Test identity map with a tuple
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds.
-				map(new Mapper2());
-
-		List<Tuple3<Integer, Long, String>> result = identityMapDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n" +
-				"6,3,Luke Skywalker\n" +
-				"7,4,Comment#1\n" +
-				"8,4,Comment#2\n" +
-				"9,4,Comment#3\n" +
-				"10,4,Comment#4\n" +
-				"11,5,Comment#5\n" +
-				"12,5,Comment#6\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" +
-				"15,5,Comment#9\n" +
-				"16,6,Comment#10\n" +
-				"17,6,Comment#11\n" +
-				"18,6,Comment#12\n" +
-				"19,6,Comment#13\n" +
-				"20,6,Comment#14\n" +
-				"21,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Mapper2 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
-				throws Exception {
-			return value;
-		}
-	}
-
-	@Test
-	public void testTypeConversionMapperCustomToTuple() throws Exception {
-		/*
-		 * Test type conversion mapper (Custom -> Tuple)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> typeConversionMapDs = ds.
-				map(new Mapper3());
-
-		List<Tuple3<Integer, Long, String>> result = typeConversionMapDs.collect();
-
-		String expected = "1,0,Hi\n" +
-				"2,1,Hello\n" +
-				"2,2,Hello world\n" +
-				"3,3,Hello world, how are you?\n" +
-				"3,4,I am fine.\n" +
-				"3,5,Luke Skywalker\n" +
-				"4,6,Comment#1\n" +
-				"4,7,Comment#2\n" +
-				"4,8,Comment#3\n" +
-				"4,9,Comment#4\n" +
-				"5,10,Comment#5\n" +
-				"5,11,Comment#6\n" +
-				"5,12,Comment#7\n" +
-				"5,13,Comment#8\n" +
-				"5,14,Comment#9\n" +
-				"6,15,Comment#10\n" +
-				"6,16,Comment#11\n" +
-				"6,17,Comment#12\n" +
-				"6,18,Comment#13\n" +
-				"6,19,Comment#14\n" +
-				"6,20,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Mapper3 implements MapFunction<CustomType, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
-
-		@Override
-		public Tuple3<Integer, Long, String> map(CustomType value) throws Exception {
-			out.setField(value.myInt, 0);
-			out.setField(value.myLong, 1);
-			out.setField(value.myString, 2);
-			return out;
-		}
-	}
-
-	@Test
-	public void testTypeConversionMapperTupleToBasic() throws Exception {
-		/*
-		 * Test type conversion mapper (Tuple -> Basic)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<String> typeConversionMapDs = ds.
-				map(new Mapper4());
-
-		List<String> result = typeConversionMapDs.collect();
-
-		String expected = "Hi\n" + "Hello\n" + "Hello world\n" +
-				"Hello world, how are you?\n" +
-				"I am fine.\n" + "Luke Skywalker\n" +
-				"Comment#1\n" +	"Comment#2\n" +
-				"Comment#3\n" +	"Comment#4\n" +
-				"Comment#5\n" +	"Comment#6\n" +
-				"Comment#7\n" + "Comment#8\n" +
-				"Comment#9\n" +	"Comment#10\n" +
-				"Comment#11\n" + "Comment#12\n" +
-				"Comment#13\n" + "Comment#14\n" +
-				"Comment#15\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper4 implements MapFunction<Tuple3<Integer, Long, String>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map(Tuple3<Integer, Long, String> value) throws Exception {
-			return value.getField(2);
-		}
-	}
-
-	@Test
-	public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() throws
-	Exception {
-		/*
-		 * Test mapper on tuple - Increment Integer field, reorder second and third fields
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, String, Long>> tupleMapDs = ds.
-				map(new Mapper5());
-
-		List<Tuple3<Integer, String, Long>> result = tupleMapDs.collect();
-
-		String expected = "2,Hi,1\n" +
-				"3,Hello,2\n" +
-				"4,Hello world,2\n" +
-				"5,Hello world, how are you?,3\n" +
-				"6,I am fine.,3\n" +
-				"7,Luke Skywalker,3\n" +
-				"8,Comment#1,4\n" +
-				"9,Comment#2,4\n" +
-				"10,Comment#3,4\n" +
-				"11,Comment#4,4\n" +
-				"12,Comment#5,5\n" +
-				"13,Comment#6,5\n" +
-				"14,Comment#7,5\n" +
-				"15,Comment#8,5\n" +
-				"16,Comment#9,5\n" +
-				"17,Comment#10,6\n" +
-				"18,Comment#11,6\n" +
-				"19,Comment#12,6\n" +
-				"20,Comment#13,6\n" +
-				"21,Comment#14,6\n" +
-				"22,Comment#15,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Mapper5 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, String, Long>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, String, Long> out = new Tuple3<Integer, String, Long>();
-
-		@Override
-		public Tuple3<Integer, String, Long> map(Tuple3<Integer, Long, String> value)
-				throws Exception {
-			Integer incr = Integer.valueOf(value.f0.intValue() + 1);
-			out.setFields(incr, value.f2, value.f1);
-			return out;
-		}
-	}
-
-	@Test
-	public void testMapperOnCustomLowercaseString() throws Exception {
-		/*
-		 * Test mapper on Custom - lowercase myString
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> customMapDs = ds.
-				map(new Mapper6());
-
-		List<CustomType> result = customMapDs.collect();
-
-		String expected = "1,0,hi\n" +
-				"2,1,hello\n" +
-				"2,2,hello world\n" +
-				"3,3,hello world, how are you?\n" +
-				"3,4,i am fine.\n" +
-				"3,5,luke skywalker\n" +
-				"4,6,comment#1\n" +
-				"4,7,comment#2\n" +
-				"4,8,comment#3\n" +
-				"4,9,comment#4\n" +
-				"5,10,comment#5\n" +
-				"5,11,comment#6\n" +
-				"5,12,comment#7\n" +
-				"5,13,comment#8\n" +
-				"5,14,comment#9\n" +
-				"6,15,comment#10\n" +
-				"6,16,comment#11\n" +
-				"6,17,comment#12\n" +
-				"6,18,comment#13\n" +
-				"6,19,comment#14\n" +
-				"6,20,comment#15\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper6 implements MapFunction<CustomType, CustomType> {
-		private static final long serialVersionUID = 1L;
-		private final CustomType out = new CustomType();
-
-		@Override
-		public CustomType map(CustomType value) throws Exception {
-			out.myInt = value.myInt;
-			out.myLong = value.myLong;
-			out.myString = value.myString.toLowerCase();
-			return out;
-		}
-	}
-
-	@Test
-	public void test() throws Exception {
-		/*
-		 * Test mapper if UDF returns input object - increment first field of a tuple
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> inputObjMapDs = ds.
-				map(new Mapper7());
-
-		List<Tuple3<Integer, Long, String>> result = inputObjMapDs.collect();
-
-		String expected = "2,1,Hi\n" +
-				"3,2,Hello\n" +
-				"4,2,Hello world\n" +
-				"5,3,Hello world, how are you?\n" +
-				"6,3,I am fine.\n" +
-				"7,3,Luke Skywalker\n" +
-				"8,4,Comment#1\n" +
-				"9,4,Comment#2\n" +
-				"10,4,Comment#3\n" +
-				"11,4,Comment#4\n" +
-				"12,5,Comment#5\n" +
-				"13,5,Comment#6\n" +
-				"14,5,Comment#7\n" +
-				"15,5,Comment#8\n" +
-				"16,5,Comment#9\n" +
-				"17,6,Comment#10\n" +
-				"18,6,Comment#11\n" +
-				"19,6,Comment#12\n" +
-				"20,6,Comment#13\n" +
-				"21,6,Comment#14\n" +
-				"22,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Mapper7 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
-				throws Exception {
-			Integer incr = Integer.valueOf(value.f0.intValue() + 1);
-			value.setField(incr, 0);
-			return value;
-		}
-	}
-
-	@Test
-	public void testMapWithBroadcastSet() throws Exception {
-		/*
-		 * Test map with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> bcMapDs = ds.
-				map(new RichMapper1()).withBroadcastSet(ints, "ints");
-		List<Tuple3<Integer, Long, String>> result = bcMapDs.collect();
-
-		String expected = "55,1,Hi\n" +
-				"55,2,Hello\n" +
-				"55,2,Hello world\n" +
-				"55,3,Hello world, how are you?\n" +
-				"55,3,I am fine.\n" +
-				"55,3,Luke Skywalker\n" +
-				"55,4,Comment#1\n" +
-				"55,4,Comment#2\n" +
-				"55,4,Comment#3\n" +
-				"55,4,Comment#4\n" +
-				"55,5,Comment#5\n" +
-				"55,5,Comment#6\n" +
-				"55,5,Comment#7\n" +
-				"55,5,Comment#8\n" +
-				"55,5,Comment#9\n" +
-				"55,6,Comment#10\n" +
-				"55,6,Comment#11\n" +
-				"55,6,Comment#12\n" +
-				"55,6,Comment#13\n" +
-				"55,6,Comment#14\n" +
-				"55,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichMapper1 extends RichMapFunction<Tuple3<Integer,Long,String>,
-	Tuple3<Integer,	Long,String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
-		private Integer f2Replace = 0;
-
-		@Override
-		public void open(Configuration config) {
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			f2Replace = sum;
-		}
-
-		@Override
-		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
-				throws Exception {
-			out.setFields(f2Replace, value.f1, value.f2);
-			return out;
-		}
-	}
-
-	static final String testKey = "testVariable";
-	static final int testValue = 666;
-
-	@Test
-	public void testPassingConfigurationObject() throws Exception {
-		/*
-		 * Test passing configuration object.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		Configuration conf = new Configuration();
-		conf.setInteger(testKey, testValue);
-		DataSet<Tuple3<Integer, Long, String>> bcMapDs = ds.
-				map(new RichMapper2()).withParameters(conf);
-		List<Tuple3<Integer, Long, String>> result = bcMapDs.collect();
-
-		String expected = "1,1,Hi\n"
-				+ "2,2,Hello\n"
-				+ "3,2,Hello world";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichMapper2 extends RichMapFunction<Tuple3<Integer,Long,String>,
-	Tuple3<Integer,	Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void open(Configuration config) {
-			int val = config.getInteger(testKey, -1);
-			Assert.assertEquals(testValue, val);
-		}
-
-		@Override
-		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value) {
-			return value;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java
deleted file mode 100644
index cc895c2..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/MapPartitionITCase.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.javaApiOperators;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.MapPartitionFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.util.Collector;
-
-@SuppressWarnings("serial")
-public class MapPartitionITCase extends JavaProgramTestBase {
-
-	private static final String IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n"
-			+ "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n"
-			+ "5 9\n7 7\n8 8\n1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n";
-
-	private static final String RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n";
-	
-	
-	private List<Tuple2<String, String>> input = new ArrayList<Tuple2<String,String>>();
-	
-	private List<Tuple2<String, Integer>> expected = new ArrayList<Tuple2<String,Integer>>();
-	
-	private List<Tuple2<String, Integer>> result = new ArrayList<Tuple2<String,Integer>>();
-	
-
-	@Override
-	protected void preSubmit() throws Exception {
-
-		// create input
-		for (String s :IN.split("\n")) {
-			String[] fields = s.split(" ");
-			input.add(new Tuple2<String, String>(fields[0], fields[1]));
-		}
-		
-		// create expected
-		for (String s : RESULT.split("\n")) {
-			String[] fields = s.split(" ");
-			expected.add(new Tuple2<String, Integer>(fields[0], Integer.parseInt(fields[1])));
-		}
-		
-	}
-	
-	@Override
-	protected void postSubmit() {
-		compareResultCollections(expected, result, new TestBaseUtils.TupleComparator<Tuple2<String, Integer>>());
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		DataSet<Tuple2<String, String>> data = env.fromCollection(input);
-		
-		data.mapPartition(new TestMapPartition()).output(new LocalCollectionOutputFormat<Tuple2<String,Integer>>(result));
-		
-		env.execute();
-	}
-	
-	
-	public static class TestMapPartition implements MapPartitionFunction<Tuple2<String, String>, Tuple2<String, Integer>> {
-
-		@Override
-		public void mapPartition(Iterable<Tuple2<String, String>> values, Collector<Tuple2<String, Integer>> out) {
-			for (Tuple2<String, String> value : values) {
-				String keyString = value.f0;
-				String valueString = value.f1;
-				
-				int keyInt = Integer.parseInt(keyString);
-				int valueInt = Integer.parseInt(valueString);
-
-				if (keyInt + valueInt < 10) {
-					out.collect(new Tuple2<String, Integer>(valueString, keyInt + 10));
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java
deleted file mode 100644
index 63abe63..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ObjectReuseITCase.java
+++ /dev/null
@@ -1,216 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * These check whether the object-reuse execution mode does really reuse objects.
- */
-@SuppressWarnings("serial" )
-@RunWith(Parameterized.class)
-public class ObjectReuseITCase extends MultipleProgramsTestBase {
-
-	private static final List<Tuple2<String, Integer>> REDUCE_DATA =
-		Arrays.asList(
-			new Tuple2<>("a", 1), new Tuple2<>("a", 2),
-			new Tuple2<>("a", 3), new Tuple2<>("a", 4),
-			new Tuple2<>("a", 50));
-
-	private static final List<Tuple2<String, Integer>> GROUP_REDUCE_DATA =
-		Arrays.asList(
-			new Tuple2<>("a", 1), new Tuple2<>("a", 2),
-			new Tuple2<>("a", 3), new Tuple2<>("a", 4),
-			new Tuple2<>("a", 5));
-	
-	
-	private final boolean objectReuse;
-	
-	public ObjectReuseITCase(boolean objectReuse) {
-		super(TestExecutionMode.CLUSTER);
-		this.objectReuse = objectReuse;
-	}
-
-	@Test
-	public void testKeyedReduce() throws Exception {
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		if (objectReuse) {
-			env.getConfig().enableObjectReuse();
-		} else {
-			env.getConfig().disableObjectReuse();
-		}
-
-		DataSet<Tuple2<String, Integer>> input = env.fromCollection(REDUCE_DATA);
-		
-		DataSet<Tuple2<String, Integer>> result = input
-			.groupBy(0)
-			.reduce(new ReduceFunction<Tuple2<String, Integer>>() {
-
-				@Override
-				public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) {
-					value2.f1 += value1.f1;
-					return value2;
-				}
-			});
-
-		Tuple2<String, Integer> res = result.collect().get(0);
-		assertEquals(new Tuple2<>("a", 60), res);
-	}
-
-	@Test
-	public void testGlobalReduce() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		if (objectReuse) {
-			env.getConfig().enableObjectReuse();
-		} else {
-			env.getConfig().disableObjectReuse();
-		}
-
-		DataSet<Tuple2<String, Integer>> input = env.fromCollection(REDUCE_DATA);
-
-		DataSet<Tuple2<String, Integer>> result = input.reduce(
-			new ReduceFunction<Tuple2<String, Integer>>() {
-
-				@Override
-				public Tuple2<String, Integer> reduce(
-						Tuple2<String, Integer> value1,
-						Tuple2<String, Integer> value2) {
-					
-					if (value1.f1 % 3 == 0) {
-						value1.f1 += value2.f1;
-						return value1;
-					} else {
-						value2.f1 += value1.f1;
-						return value2;
-					}
-				}
-
-			});
-
-		Tuple2<String, Integer> res = result.collect().get(0);
-		assertEquals(new Tuple2<>("a", 60), res);
-	}
-
-	@Test
-	public void testKeyedGroupReduce() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		if (objectReuse) {
-			env.getConfig().enableObjectReuse();
-		} else {
-			env.getConfig().disableObjectReuse();
-		}
-
-		DataSet<Tuple2<String, Integer>> input = env.fromCollection(GROUP_REDUCE_DATA);
-
-		DataSet<Tuple2<String, Integer>> result = input.groupBy(0).reduceGroup(
-			new GroupReduceFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
-
-				@Override
-				public void reduce(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
-					List<Tuple2<String, Integer>> list = new ArrayList<>();
-					for (Tuple2<String, Integer> val : values) {
-						list.add(val);
-					}
-
-					for (Tuple2<String, Integer> val : list) {
-						out.collect(val);
-					}
-				}
-			});
-
-		List<Tuple2<String, Integer>> is = result.collect();
-		Collections.sort(is, new TupleComparator<Tuple2<String, Integer>>());
-
-		List<Tuple2<String, Integer>> expected = env.getConfig().isObjectReuseEnabled() ?
-			Arrays.asList(new Tuple2<>("a", 4), new Tuple2<>("a", 4),
-				new Tuple2<>("a", 5), new Tuple2<>("a", 5), new Tuple2<>("a", 5)) :
-			Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2),
-				new Tuple2<>("a", 3), new Tuple2<>("a", 4), new Tuple2<>("a", 5));
-
-		assertEquals(expected, is);
-	}
-	
-	@Test
-	public void testGlobalGroupReduce() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		if (objectReuse) {
-			env.getConfig().enableObjectReuse();
-		} else {
-			env.getConfig().disableObjectReuse();
-		}
-
-		DataSet<Tuple2<String, Integer>> input = env.fromCollection(GROUP_REDUCE_DATA);
-		
-		DataSet<Tuple2<String, Integer>> result = input.reduceGroup(
-			new GroupReduceFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
-
-				@Override
-				public void reduce(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
-					List<Tuple2<String, Integer>> list = new ArrayList<>();
-					for (Tuple2<String, Integer> val : values) {
-						list.add(val);
-					}
-	
-					for (Tuple2<String, Integer> val : list) {
-						out.collect(val);
-					}
-				}
-			});
-
-		List<Tuple2<String, Integer>> is = result.collect();
-		Collections.sort(is, new TupleComparator<Tuple2<String, Integer>>());
-
-		List<Tuple2<String, Integer>> expected = env.getConfig().isObjectReuseEnabled() ?
-			Arrays.asList(new Tuple2<>("a", 4), new Tuple2<>("a", 4),
-				new Tuple2<>("a", 5), new Tuple2<>("a", 5), new Tuple2<>("a", 5)) :
-			Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2),
-				new Tuple2<>("a", 3), new Tuple2<>("a", 4), new Tuple2<>("a", 5));
-		
-		assertEquals(expected, is);
-	}
-
-	@Parameterized.Parameters(name = "Execution mode = CLUSTER, Reuse = {0}")
-	public static Collection<Object[]> executionModes() {
-		return Arrays.asList(
-			new Object[] { false, },
-			new Object[] { true } );
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
deleted file mode 100644
index 5215a36..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/OuterJoinITCase.java
+++ /dev/null
@@ -1,680 +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.javaApiOperators;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.RichFlatJoinFunction;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.CompilerException;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Collection;
-import java.util.List;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class OuterJoinITCase extends MultipleProgramsTestBase {
-
-	public OuterJoinITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Test
-	public void testLeftOuterJoin1() throws Exception {
-		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
-	}
-
-	@Test
-	public void testLeftOuterJoin2() throws Exception {
-		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
-	}
-
-	@Test
-	public void testLeftOuterJoin3() throws Exception {
-		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
-	}
-
-	@Test
-	public void testLeftOuterJoin4() throws Exception {
-		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
-	}
-
-	@Test (expected = InvalidProgramException.class)
-	public void testLeftOuterJoin5() throws Exception {
-		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
-	}
-
-	private void testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
-		/*
-		 * UDF Join on tuples with key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.leftOuterJoin(ds2, hint)
-						.where(0)
-						.equalTo(0)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello,Hallo Welt wie\n" +
-				"Hello world,null\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testRightOuterJoin1() throws Exception {
-		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
-	}
-
-	@Test
-	public void testRightOuterJoin2() throws Exception {
-		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
-	}
-
-	@Test
-	public void testRightOuterJoin3() throws Exception {
-		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
-	}
-
-	@Test
-	public void testRightOuterJoin4() throws Exception {
-		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
-	}
-
-	@Test (expected = InvalidProgramException.class)
-	public void testRightOuterJoin5() throws Exception {
-		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
-	}
-
-	private void testRightOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
-		/*
-		 * UDF Join on tuples with key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.rightOuterJoin(ds2, hint)
-						.where(1)
-						.equalTo(1)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"null,Hallo Welt wie\n" +
-				"Hello world,Hallo Welt\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testFullOuterJoin1() throws Exception {
-		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
-	}
-
-	@Test
-	public void testFullOuterJoin2() throws Exception {
-		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
-	}
-
-	@Test
-	public void testFullOuterJoin3() throws Exception {
-		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
-	}
-
-	@Test (expected = InvalidProgramException.class)
-	public void testFullOuterJoin4() throws Exception {
-		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
-	}
-
-	@Test (expected = InvalidProgramException.class)
-	public void testFullOuterJoin5() throws Exception {
-		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
-	}
-
-	private void testFullOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
-		/*
-		 * UDF Join on tuples with key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.fullOuterJoin(ds2, hint)
-						.where(0)
-						.equalTo(2)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "null,Hallo\n" +
-				"Hi,Hallo Welt\n" +
-				"Hello,Hallo Welt wie\n" +
-				"Hello world,null\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinOnTuplesWithCompositeKeyPositions() throws Exception {
-		/*
-		 * UDF Join on tuples with multiple key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(0, 1)
-						.equalTo(0, 4)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,null\n" +
-				"null,Hallo Welt wie\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithBroadcastSet() throws Exception {
-		/*
-		 * Join with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple3<String, String, Integer>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(1)
-						.equalTo(4)
-						.with(new T3T5BCJoin())
-						.withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<String, String, Integer>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo,55\n" +
-				"Hi,Hallo Welt wie,55\n" +
-				"Hello,Hallo Welt,55\n" +
-				"Hello world,Hallo Welt,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithMixedKeyTypes1() throws Exception {
-		/*
-		 * Join on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(new KeySelector1())
-						.equalTo(0)
-						.with(new CustT3Join());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hi\n" +
-				"Hello,Hello\n" +
-				"Hello world,Hello\n" +
-				"null,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	public static class KeySelector1 implements KeySelector<CustomType, Integer> {
-		@Override
-		public Integer getKey(CustomType value) {
-			return value.myInt;
-		}
-	}
-
-
-	@Test
-	public void testJoinWithMixedKeyTypes2()
-			throws Exception {
-		/*
-		 * Join on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(1)
-						.equalTo(new KeySelector2())
-						.with(new T3CustJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "null,Hi\n" +
-				"Hi,Hello\n" +
-				"Hello,Hello world\n" +
-				"Hello world,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<CustomType, Long> {
-		@Override
-		public Long getKey(CustomType value) {
-			return value.myLong;
-		}
-	}
-
-	@Test
-	public void testJoinWithTupleReturningKeySelectors() throws Exception {
-		/*
-		 * UDF Join on tuples with tuple-returning key selectors
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(new KeySelector3()) //0, 1
-						.equalTo(new KeySelector4()) // 0, 4
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,null\n" +
-				"null,Hallo Welt wie\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector3 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
-			return new Tuple2<>(t.f0, t.f1);
-		}
-	}
-
-	public static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
-			return new Tuple2<>(t.f0, t.f4);
-		}
-	}
-
-	@Test
-	public void testJoinWithNestedKeyExpression1() throws Exception {
-		/*
-		 * Join nested pojo against tuple (selected using a string)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("nestedPojo.longNumber")
-						.equalTo("f6")
-						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
-
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithNestedKeyExpression2() throws Exception {
-		/*
-		 * Join nested pojo against tuple (selected as an integer)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("nestedPojo.longNumber")
-						.equalTo(6) // <--- difference!
-						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
-
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithCompositeKeyExpressions() throws Exception {
-		/*
-		 * selecting multiple fields using expression language
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("nestedPojo.longNumber", "number", "str")
-						.equalTo("f6", "f0", "f1")
-						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedIntoTuple() throws Exception {
-		/*
-		 * nested into tuple
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("nestedPojo.longNumber", "number", "nestedTupleWithCustom.f0")
-						.equalTo("f6", "f0", "f2")
-						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedIntoTupleIntoPojo() throws Exception {
-		/*
-		 * nested into tuple into pojo
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("nestedTupleWithCustom.f0", "nestedTupleWithCustom.f1.myInt", "nestedTupleWithCustom.f1.myLong")
-						.equalTo("f2", "f3", "f4")
-						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNonPojoToVerifyFullTupleKeys() throws Exception {
-		/*
-		 * Non-POJO test to verify that full-tuple keys are working.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where(0)
-						.equalTo("f0.f0", "f0.f1") // key is now Tuple2<Integer, Integer>
-						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
-
-		env.setParallelism(1);
-		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
-
-		String expected = "((1,1),one),((1,1),one)\n" +
-				"((2,2),two),((2,2),two)\n" +
-				"((3,3),three),((3,3),three)\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	@Test
-	public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception {
-		/*
-		 * Non-POJO test to verify "nested" tuple-element selection.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("f0.f0")
-						.equalTo("f0.f0") // key is now Integer from Tuple2<Integer, Integer>
-						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
-
-		env.setParallelism(1);
-		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
-
-		String expected = "((1,1),one),((1,1),one)\n" +
-				"((2,2),two),((2,2),two)\n" +
-				"((3,3),three),((3,3),three)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testFullPojoWithFullTuple() throws Exception {
-		/*
-		 * full pojo with full tuple
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env);
-		DataSet<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> joinDs =
-				ds1.fullOuterJoin(ds2)
-						.where("*")
-						.equalTo("*")
-						.with(new ProjectBothFunction<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>());
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
-				"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +
-				"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithAtomicType1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Integer> ds2 = env.fromElements(1, 2);
-
-		DataSet<Tuple2<Tuple3<Integer, Long, String>, Integer>> joinDs = ds1
-				.fullOuterJoin(ds2)
-				.where(0)
-				.equalTo("*")
-				.with(new ProjectBothFunction<Tuple3<Integer, Long, String>, Integer>())
-				.returns("Tuple2<java.lang.Object,java.lang.Object>");
-
-		List<Tuple2<Tuple3<Integer, Long, String>, Integer>> result = joinDs.collect();
-
-		String expected = "(1,1,Hi),1\n" +
-				"(2,2,Hello),2\n" +
-				"(3,2,Hello world),null\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithAtomicType2() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ds1 = env.fromElements(1, 2);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-
-		DataSet<Tuple2<Integer, Tuple3<Integer, Long, String>>> joinDs = ds1
-				.fullOuterJoin(ds2)
-				.where("*")
-				.equalTo(0)
-				.with(new ProjectBothFunction<Integer, Tuple3<Integer, Long, String>>())
-				.returns("Tuple2<java.lang.Object,java.lang.Object>");
-
-
-		List<Tuple2<Integer, Tuple3<Integer, Long, String>>> result = joinDs.collect();
-
-		String expected = "1,(1,1,Hi)\n" +
-				"2,(2,2,Hello)\n" +
-				"null,(3,2,Hello world)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class T3T5FlatJoin implements FlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<String, String>> {
-
-		@Override
-		public void join(Tuple3<Integer, Long, String> first,
-				Tuple5<Integer, Long, Integer, String, Long> second,
-				Collector<Tuple2<String, String>> out) {
-
-			out.collect(new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.f3));
-		}
-
-	}
-
-	public static class T3T5BCJoin extends RichFlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<String, String, Integer>> {
-
-		private int broadcast;
-
-		@Override
-		public void open(Configuration config) {
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for (Integer i : ints) {
-				sum += i;
-			}
-			broadcast = sum;
-		}
-
-		@Override
-		public void join(Tuple3<Integer, Long, String> first, Tuple5<Integer, Long, Integer, String, Long> second,
-				Collector<Tuple3<String, String, Integer>> out) throws Exception {
-			out.collect(new Tuple3<>(first == null ? null : first.f2, second == null ? null : second.f3, broadcast));
-		}
-	}
-
-	public static class T3CustJoin implements JoinFunction<Tuple3<Integer, Long, String>, CustomType, Tuple2<String, String>> {
-
-		@Override
-		public Tuple2<String, String> join(Tuple3<Integer, Long, String> first,
-				CustomType second) {
-
-			return new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.myString);
-		}
-	}
-
-	public static class CustT3Join implements JoinFunction<CustomType, Tuple3<Integer, Long, String>, Tuple2<String, String>> {
-
-		@Override
-		public Tuple2<String, String> join(CustomType first, Tuple3<Integer, Long, String> second) {
-
-			return new Tuple2<>(first == null ? null : first.myString, second == null ? null : second.f2);
-		}
-	}
-
-	/**
-	 * Deliberately untyped join function, which emits a Tuple2 of the left and right side.
-	 */
-	public static class ProjectBothFunction<IN1, IN2> implements JoinFunction<IN1, IN2, Tuple2<IN1, IN2>> {
-		@Override
-		public Tuple2<IN1, IN2> join(IN1 first, IN2 second) throws Exception {
-			return new Tuple2<>(first, second);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
deleted file mode 100644
index 85d70e3..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java
+++ /dev/null
@@ -1,848 +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.javaApiOperators;
-
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.MapPartitionFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.AggregateOperator;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@RunWith(Parameterized.class)
-@SuppressWarnings("serial")
-public class PartitionITCase extends MultipleProgramsTestBase {
-
-	public PartitionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testHashPartitionByKeyField() throws Exception {
-		/*
-		 * Test hash partition by key field
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-				.partitionByHash(1)
-				.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-				"2\n" +
-				"3\n" +
-				"4\n" +
-				"5\n" +
-				"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRangePartitionByKeyField() throws Exception {
-		/*
-		 * Test range partition by key field
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-			.partitionByRange(1)
-			.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-			"2\n" +
-			"3\n" +
-			"4\n" +
-			"5\n" +
-			"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testHashPartitionByKeyField2() throws Exception {
-		/*
-		 * Test hash partition by key field
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		AggregateOperator<Tuple3<Integer, Long, String>> sum = ds
-			.map(new PrefixMapper())
-			.partitionByHash(1, 2)
-			.groupBy(1, 2)
-			.sum(0);
-
-		List<Tuple3<Integer, Long, String>> result = sum.collect();
-
-		String expected = "(1,1,Hi)\n" +
-			"(5,2,Hello)\n" +
-			"(4,3,Hello)\n" +
-			"(5,3,I am )\n" +
-			"(6,3,Luke )\n" +
-			"(34,4,Comme)\n" +
-			"(65,5,Comme)\n" +
-			"(111,6,Comme)";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRangePartitionByKeyField2() throws Exception {
-		/*
-		 * Test range partition by key field
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		AggregateOperator<Tuple3<Integer, Long, String>> sum = ds
-			.map(new PrefixMapper())
-			.partitionByRange(1, 2)
-			.groupBy(1, 2)
-			.sum(0);
-
-		List<Tuple3<Integer, Long, String>> result = sum.collect();
-
-		String expected = "(1,1,Hi)\n" +
-		"(5,2,Hello)\n" +
-		"(4,3,Hello)\n" +
-		"(5,3,I am )\n" +
-		"(6,3,Luke )\n" +
-		"(34,4,Comme)\n" +
-		"(65,5,Comme)\n" +
-		"(111,6,Comme)";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testHashPartitionOfAtomicType() throws Exception {
-		/*
-		 * Test hash partition of atomic type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Long> uniqLongs = env.generateSequence(1, 6)
-			.union(env.generateSequence(1, 6))
-			.rebalance()
-			.partitionByHash("*")
-			.mapPartition(new UniqueLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-			"2\n" +
-			"3\n" +
-			"4\n" +
-			"5\n" +
-			"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRangePartitionOfAtomicType() throws Exception {
-		/*
-		 * Test range partition of atomic type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Long> uniqLongs = env.generateSequence(1, 6)
-			.union(env.generateSequence(1, 6))
-			.rebalance()
-			.partitionByRange("*")
-			.mapPartition(new UniqueLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-			"2\n" +
-			"3\n" +
-			"4\n" +
-			"5\n" +
-			"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testHashPartitionByKeySelector() throws Exception {
-		/*
-		 * Test hash partition by key selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-				.partitionByHash(new KeySelector1())
-				.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-				"2\n" +
-				"3\n" +
-				"4\n" +
-				"5\n" +
-				"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	private static class PrefixMapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		@Override
-		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value) throws Exception {
-			if (value.f2.length() > 5) {
-				value.f2 = value.f2.substring(0, 5);
-			}
-			return value;
-		}
-	}
-
-	@Test
-	public void testRangePartitionByKeySelector() throws Exception {
-		/*
-		 * Test range partition by key selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-			.partitionByRange(new KeySelector1())
-			.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-			"2\n" +
-			"3\n" +
-			"4\n" +
-			"5\n" +
-			"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector1 implements KeySelector<Tuple3<Integer,Long,String>, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long getKey(Tuple3<Integer, Long, String> value) throws Exception {
-			return value.f1;
-		}
-
-	}
-
-	@Test
-	public void testForcedRebalancing() throws Exception {
-		/*
-		 * Test forced rebalancing
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// generate some number in parallel
-		DataSet<Long> ds = env.generateSequence(1,3000);
-		DataSet<Tuple2<Integer, Integer>> uniqLongs = ds
-				// introduce some partition skew by filtering
-				.filter(new Filter1())
-				// rebalance
-				.rebalance()
-				// count values in each partition
-				.map(new PartitionIndexMapper())
-				.groupBy(0)
-				.reduce(new Reducer1())
-				// round counts to mitigate runtime scheduling effects (lazy split assignment)
-				.map(new Mapper1());
-
-		List<Tuple2<Integer, Integer>> result = uniqLongs.collect();
-
-		StringBuilder expected = new StringBuilder();
-		int numPerPartition = 2220 / env.getParallelism() / 10;
-		for (int i = 0; i < env.getParallelism(); i++) {
-			expected.append('(').append(i).append(',')
-			.append(numPerPartition).append(")\n");
-		}
-
-		compareResultAsText(result, expected.toString());
-	}
-
-	public static class Filter1 implements FilterFunction<Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Long value) throws Exception {
-			return value > 780;
-		}
-	}
-
-	public static class Reducer1 implements ReduceFunction<Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> v1, Tuple2<Integer, Integer> v2) {
-			return new Tuple2<>(v1.f0, v1.f1+v2.f1);
-		}
-	}
-
-	public static class Mapper1 implements MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer,
-	Integer>>{
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) throws Exception {
-			value.f1 = (value.f1 / 10);
-			return value;
-		}
-
-	}
-
-	@Test
-	public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exception {
-		/*
-		 * Test hash partition by key field and different parallelism
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-				.partitionByHash(1).setParallelism(4)
-				.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-				"2\n" +
-				"3\n" +
-				"4\n" +
-				"5\n" +
-				"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRangePartitionByKeyFieldAndDifferentParallelism() throws Exception {
-		/*
-		 * Test range partition by key field and different parallelism
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Long> uniqLongs = ds
-			.partitionByRange(1).setParallelism(4)
-			.mapPartition(new UniqueTupleLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "1\n" +
-			"2\n" +
-			"3\n" +
-			"4\n" +
-			"5\n" +
-			"6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testHashPartitionWithKeyExpression() throws Exception {
-		/*
-		 * Test hash partition with key expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
-		DataSet<Long> uniqLongs = ds
-				.partitionByHash("nestedPojo.longNumber").setParallelism(4)
-				.mapPartition(new UniqueNestedPojoLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "10000\n" +
-				"20000\n" +
-				"30000\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testRangePartitionWithKeyExpression() throws Exception {
-		/*
-		 * Test range partition with key expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
-		DataSet<Long> uniqLongs = ds
-			.partitionByRange("nestedPojo.longNumber").setParallelism(4)
-			.mapPartition(new UniqueNestedPojoLongMapper());
-		List<Long> result = uniqLongs.collect();
-
-		String expected = "10000\n" +
-			"20000\n" +
-			"30000\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class UniqueTupleLongMapper implements MapPartitionFunction<Tuple3<Integer,Long,String>, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void mapPartition(Iterable<Tuple3<Integer, Long, String>> records, Collector<Long> out) throws Exception {
-			HashSet<Long> uniq = new HashSet<>();
-			for(Tuple3<Integer,Long,String> t : records) {
-				uniq.add(t.f1);
-			}
-			for(Long l : uniq) {
-				out.collect(l);
-			}
-		}
-	}
-
-	public static class UniqueLongMapper implements MapPartitionFunction<Long, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void mapPartition(Iterable<Long> longs, Collector<Long> out) throws Exception {
-			HashSet<Long> uniq = new HashSet<>();
-			for(Long l : longs) {
-				uniq.add(l);
-			}
-			for(Long l : uniq) {
-				out.collect(l);
-			}
-		}
-	}
-
-	public static class UniqueNestedPojoLongMapper implements MapPartitionFunction<POJO, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void mapPartition(Iterable<POJO> records, Collector<Long> out) throws Exception {
-			HashSet<Long> uniq = new HashSet<>();
-			for(POJO t : records) {
-				uniq.add(t.nestedPojo.longNumber);
-			}
-			for(Long l : uniq) {
-				out.collect(l);
-			}
-		}
-	}
-
-	public static class PartitionIndexMapper extends RichMapFunction<Long, Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> map(Long value) throws Exception {
-			return new Tuple2<>(this.getRuntimeContext().getIndexOfThisSubtask(), 1);
-		}
-	}
-
-	@Test
-	public void testRangePartitionerOnSequenceData() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSource<Long> dataSource = env.generateSequence(0, 10000);
-		KeySelector<Long, Long> keyExtractor = new ObjectSelfKeySelector();
-
-		MapPartitionFunction<Long, Tuple2<Long, Long>> MinMaxSelector = new MinMaxSelector<>(new LongComparator(true));
-
-		Comparator<Tuple2<Long, Long>> tuple2Comparator = new Tuple2Comparator(new LongComparator(true));
-
-		List<Tuple2<Long, Long>> collected = dataSource.partitionByRange(keyExtractor).mapPartition(MinMaxSelector).collect();
-		Collections.sort(collected, tuple2Comparator);
-
-		long previousMax = -1;
-		for (Tuple2<Long, Long> tuple2 : collected) {
-			if (previousMax == -1) {
-				previousMax = tuple2.f1;
-			} else {
-				long currentMin = tuple2.f0;
-				assertTrue(tuple2.f0 < tuple2.f1);
-				assertEquals(previousMax + 1, currentMin);
-				previousMax = tuple2.f1;
-			}
-		}
-	}
-
-	@Test(expected = InvalidProgramException.class)
-	public void testRangePartitionInIteration() throws Exception {
-
-		// does not apply for collection execution
-		if (super.mode == TestExecutionMode.COLLECTION) {
-			throw new InvalidProgramException("Does not apply for collection execution");
-		}
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSource<Long> source = env.generateSequence(0, 10000);
-
-		DataSet<Tuple2<Long, String>> tuples = source.map(new MapFunction<Long, Tuple2<Long, String>>() {
-			@Override
-			public Tuple2<Long, String> map(Long v) throws Exception {
-				return new Tuple2<>(v, Long.toString(v));
-			}
-		});
-
-		DeltaIteration<Tuple2<Long, String>, Tuple2<Long, String>> it = tuples.iterateDelta(tuples, 10, 0);
-		DataSet<Tuple2<Long, String>> body = it.getWorkset()
-			.partitionByRange(1) // Verify that range partition is not allowed in iteration
-			.join(it.getSolutionSet())
-			.where(0).equalTo(0).projectFirst(0).projectSecond(1);
-		DataSet<Tuple2<Long, String>> result = it.closeWith(body, body);
-
-		result.collect(); // should fail
-	}
-
-
-
-	@Test
-	public void testRangePartitionerOnSequenceDataWithOrders() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Tuple2<Long, Long>> dataSet = env.generateSequence(0, 10000)
-				.map(new MapFunction<Long, Tuple2<Long, Long>>() {
-			@Override
-			public Tuple2<Long, Long> map(Long value) throws Exception {
-				return new Tuple2<>(value / 5000, value % 5000);
-			}
-		});
-
-		final Tuple2Comparator<Long> tuple2Comparator = new Tuple2Comparator<>(new LongComparator(true),
-																			   new LongComparator(false));
-
-		MinMaxSelector<Tuple2<Long, Long>> minMaxSelector = new MinMaxSelector<>(tuple2Comparator);
-
-		final List<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>> collected = dataSet.partitionByRange(0, 1)
-				.withOrders(Order.ASCENDING, Order.DESCENDING)
-				.mapPartition(minMaxSelector)
-				.collect();
-
-		Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator));
-
-		Tuple2<Long, Long> previousMax = null;
-		for (Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> tuple2 : collected) {
-            assertTrue("Min element in each partition should be smaller than max.",
-                    tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0);
-			if (previousMax == null) {
-				previousMax = tuple2.f1;
-			} else {
-                assertTrue("Partitions overlap. Previous max should be smaller than current min.",
-                        tuple2Comparator.compare(previousMax, tuple2.f0) < 0);
-				if (previousMax.f0.equals(tuple2.f0.f0)) {
-                    //check that ordering on the second key is correct
-					assertEquals("Ordering on the second field should be continous.",
-                            previousMax.f1 - 1, tuple2.f0.f1.longValue());
-				}
-				previousMax = tuple2.f1;
-			}
-		}
-	}
-
-	@Test
-	public void testRangePartitionerOnSequenceNestedDataWithOrders() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final DataSet<Tuple2<Tuple2<Long, Long>, Long>> dataSet = env.generateSequence(0, 10000)
-				.map(new MapFunction<Long, Tuple2<Tuple2<Long, Long>, Long>>() {
-					@Override
-					public Tuple2<Tuple2<Long, Long>, Long> map(Long value) throws Exception {
-						return new Tuple2<>(new Tuple2<>(value / 5000, value % 5000), value);
-					}
-				});
-
-		final Tuple2Comparator<Long> tuple2Comparator = new Tuple2Comparator<>(new LongComparator(true),
-				new LongComparator(true));
-		MinMaxSelector<Tuple2<Long, Long>> minMaxSelector = new MinMaxSelector<>(tuple2Comparator);
-
-		final List<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>> collected = dataSet.partitionByRange(0)
-				.withOrders(Order.ASCENDING)
-				.mapPartition(new MapPartitionFunction<Tuple2<Tuple2<Long,Long>,Long>, Tuple2<Long, Long>>() {
-					@Override
-					public void mapPartition(Iterable<Tuple2<Tuple2<Long, Long>, Long>> values,
-											 Collector<Tuple2<Long, Long>> out) throws Exception {
-						for (Tuple2<Tuple2<Long, Long>, Long> value : values) {
-							out.collect(value.f0);
-						}
-					}
-				})
-				.mapPartition(minMaxSelector)
-				.collect();
-
-		Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator));
-
-		Tuple2<Long, Long> previousMax = null;
-		for (Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> tuple2 : collected) {
-            assertTrue("Min element in each partition should be smaller than max.",
-                    tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0);
-			if (previousMax == null) {
-				previousMax = tuple2.f1;
-			} else {
-                assertTrue("Partitions overlap. Previous max should be smaller than current min.",
-                        tuple2Comparator.compare(previousMax, tuple2.f0) < 0);
-				if (previousMax.f0.equals(tuple2.f0.f0)) {
-					assertEquals("Ordering on the second field should be continous.",
-                            previousMax.f1 + 1, tuple2.f0.f1.longValue());
-				}
-				previousMax = tuple2.f1;
-			}
-		}
-	}
-
-	@Test
-	public void testRangePartitionerWithKeySelectorOnSequenceNestedDataWithOrders() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		final DataSet<Tuple2<ComparablePojo, Long>> dataSet = env.generateSequence(0, 10000)
-				.map(new MapFunction<Long, Tuple2<ComparablePojo, Long>>() {
-					@Override
-					public Tuple2<ComparablePojo, Long> map(Long value) throws Exception {
-						return new Tuple2<>(new ComparablePojo(value / 5000, value % 5000), value);
-					}
-				});
-
-		final List<Tuple2<ComparablePojo, ComparablePojo>> collected = dataSet
-				.partitionByRange(new KeySelector<Tuple2<ComparablePojo, Long>, ComparablePojo>() {
-					@Override
-					public ComparablePojo getKey(Tuple2<ComparablePojo, Long> value) throws Exception {
-						return value.f0;
-					}
-				})
-				.withOrders(Order.ASCENDING)
-				.mapPartition(new MinMaxSelector<>(new ComparablePojoComparator()))
-				.mapPartition(new ExtractComparablePojo())
-				.collect();
-
-		final Comparator<Tuple2<ComparablePojo, ComparablePojo>> pojoComparator =
-				new Comparator<Tuple2<ComparablePojo, ComparablePojo>>() {
-			@Override
-			public int compare(Tuple2<ComparablePojo, ComparablePojo> o1,
-							   Tuple2<ComparablePojo, ComparablePojo> o2) {
-				return o1.f0.compareTo(o2.f1);
-			}
-		};
-		Collections.sort(collected, pojoComparator);
-
-		ComparablePojo previousMax = null;
-		for (Tuple2<ComparablePojo, ComparablePojo> element : collected) {
-			assertTrue("Min element in each partition should be smaller than max.",
-					element.f0.compareTo(element.f1) <= 0);
-			if (previousMax == null) {
-				previousMax = element.f1;
-			} else {
-				assertTrue("Partitions overlap. Previous max should be smaller than current min.",
-						previousMax.compareTo(element.f0) < 0);
-				if (previousMax.first.equals(element.f0.first)) {
-					assertEquals("Ordering on the second field should be continous.",
-							previousMax.second - 1, element.f0.second.longValue());
-				}
-				previousMax = element.f1;
-			}
-		}
-	}
-
-	private static class ExtractComparablePojo implements MapPartitionFunction<
-			Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>>,
-			Tuple2<ComparablePojo, ComparablePojo>> {
-
-		@Override
-		public void mapPartition(Iterable<Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>>> values,
-								 Collector<Tuple2<ComparablePojo, ComparablePojo>> out) throws Exception {
-			for (Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>> value : values) {
-				out.collect(new Tuple2<>(value.f0.f0, value.f1.f0));
-			}
-		}
-	}
-
-    private static class ComparablePojoComparator implements Comparator<Tuple2<ComparablePojo, Long>>, Serializable {
-
-		@Override
-		public int compare(Tuple2<ComparablePojo, Long> o1,
-						   Tuple2<ComparablePojo, Long> o2) {
-			return o1.f0.compareTo(o2.f0);
-		}
-	}
-
-	private static class ComparablePojo implements Comparable<ComparablePojo> {
-		private Long first;
-		private Long second;
-
-		public Long getFirst() {
-			return first;
-		}
-
-		public void setFirst(Long first) {
-			this.first = first;
-		}
-
-		public Long getSecond() {
-			return second;
-		}
-
-		public void setSecond(Long second) {
-			this.second = second;
-		}
-
-		public ComparablePojo(Long first,
-							  Long second) {
-			this.first = first;
-			this.second = second;
-		}
-
-		public ComparablePojo() {
-		}
-
-		@Override
-		public int compareTo(ComparablePojo o) {
-			final int firstResult = Long.compare(this.first, o.first);
-			if (firstResult == 0) {
-				return (-1) * Long.compare(this.second, o.second);
-			}
-
-			return firstResult;
-		}
-	}
-
-	private static class ObjectSelfKeySelector implements KeySelector<Long, Long> {
-		@Override
-		public Long getKey(Long value) throws Exception {
-			return value;
-		}
-	}
-
-	private static class MinMaxSelector<T> implements MapPartitionFunction<T, Tuple2<T, T>> {
-
-		private final Comparator<T> comparator;
-
-		public MinMaxSelector(Comparator<T> comparator) {
-			this.comparator = comparator;
-		}
-
-		@Override
-		public void mapPartition(Iterable<T> values, Collector<Tuple2<T, T>> out) throws Exception {
-			Iterator<T> itr = values.iterator();
-			T min = itr.next();
-			T max = min;
-			T value;
-			while (itr.hasNext()) {
-				value= itr.next();
-				if (comparator.compare(value, min) < 0) {
-					min = value;
-				}
-				if (comparator.compare(value, max) > 0) {
-					max = value;
-				}
-
-			}
-
-			Tuple2<T, T> result = new Tuple2<>(min, max);
-			out.collect(result);
-		}
-	}
-
-	private static class Tuple2Comparator<T> implements Comparator<Tuple2<T, T>>, Serializable {
-
-		private final Comparator<T> firstComparator;
-		private final Comparator<T> secondComparator;
-
-		public Tuple2Comparator(Comparator<T> comparator) {
-			this(comparator, comparator);
-		}
-
-		public Tuple2Comparator(Comparator<T> firstComparator,
-								Comparator<T> secondComparator) {
-			this.firstComparator = firstComparator;
-			this.secondComparator = secondComparator;
-		}
-
-		@Override
-		public int compare(Tuple2<T, T> first, Tuple2<T, T> second) {
-			long result = firstComparator.compare(first.f0, second.f0);
-			if (result > 0) {
-				return 1;
-			} else if (result < 0) {
-				return -1;
-			}
-
-			result = secondComparator.compare(first.f1, second.f1);
-			if (result > 0) {
-				return 1;
-			} else if (result < 0) {
-				return -1;
-			}
-
-			return 0;
-		}
-	}
-
-	private static class LongComparator implements Comparator<Long>, Serializable {
-
-		private final boolean ascending;
-
-		public LongComparator(boolean ascending) {
-			this.ascending = ascending;
-		}
-
-		@Override
-		public int compare(Long o1, Long o2) {
-			if (ascending) {
-				return Long.compare(o1, o2);
-			} else {
-				return (-1) * Long.compare(o1, o2);
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.java
deleted file mode 100644
index 1054c62..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ProjectITCase.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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-public class ProjectITCase extends JavaProgramTestBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		/*
-		 * Projection with tuple fields indexes
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<String, Long, Integer>> projDs = ds.
-				project(3,4,2);
-		List<Tuple3<String, Long, Integer>> result = projDs.collect();
-
-		String expectedResult = "Hallo,1,0\n" +
-				"Hallo Welt,2,1\n" +
-				"Hallo Welt wie,1,2\n" +
-				"Hallo Welt wie gehts?,2,3\n" +
-				"ABC,2,4\n" +
-				"BCD,3,5\n" +
-				"CDE,2,6\n" +
-				"DEF,1,7\n" +
-				"EFG,1,8\n" +
-				"FGH,2,9\n" +
-				"GHI,1,10\n" +
-				"HIJ,3,11\n" +
-				"IJK,3,12\n" +
-				"JKL,2,13\n" +
-				"KLM,2,14\n";
-
-		compareResultAsTuples(result, expectedResult);
-	}
-
-}


[05/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java
new file mode 100644
index 0000000..e2e22a8
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/util/CollectionDataSets.java
@@ -0,0 +1,772 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators.util;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+import org.apache.hadoop.io.IntWritable;
+
+import java.io.File;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import scala.math.BigInt;
+
+/**
+ * #######################################################################################################
+ * 			BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA.
+ * 			IF YOU MODIFY THE DATA MAKE SURE YOU CHECK THAT ALL TESTS ARE STILL WORKING!
+ * #######################################################################################################
+ */
+public class CollectionDataSets {
+
+	public static DataSet<Tuple3<Integer, Long, String>> get3TupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
+		data.add(new Tuple3<>(1, 1L, "Hi"));
+		data.add(new Tuple3<>(2, 2L, "Hello"));
+		data.add(new Tuple3<>(3, 2L, "Hello world"));
+		data.add(new Tuple3<>(4, 3L, "Hello world, how are you?"));
+		data.add(new Tuple3<>(5, 3L, "I am fine."));
+		data.add(new Tuple3<>(6, 3L, "Luke Skywalker"));
+		data.add(new Tuple3<>(7, 4L, "Comment#1"));
+		data.add(new Tuple3<>(8, 4L, "Comment#2"));
+		data.add(new Tuple3<>(9, 4L, "Comment#3"));
+		data.add(new Tuple3<>(10, 4L, "Comment#4"));
+		data.add(new Tuple3<>(11, 5L, "Comment#5"));
+		data.add(new Tuple3<>(12, 5L, "Comment#6"));
+		data.add(new Tuple3<>(13, 5L, "Comment#7"));
+		data.add(new Tuple3<>(14, 5L, "Comment#8"));
+		data.add(new Tuple3<>(15, 5L, "Comment#9"));
+		data.add(new Tuple3<>(16, 6L, "Comment#10"));
+		data.add(new Tuple3<>(17, 6L, "Comment#11"));
+		data.add(new Tuple3<>(18, 6L, "Comment#12"));
+		data.add(new Tuple3<>(19, 6L, "Comment#13"));
+		data.add(new Tuple3<>(20, 6L, "Comment#14"));
+		data.add(new Tuple3<>(21, 6L, "Comment#15"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<Integer, Long, String>> getSmall3TupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
+		data.add(new Tuple3<>(1, 1L, "Hi"));
+		data.add(new Tuple3<>(2, 2L, "Hello"));
+		data.add(new Tuple3<>(3, 2L, "Hello world"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple5<Integer, Long, Integer, String, Long>> get5TupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
+		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
+		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
+		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
+		data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L));
+		data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L));
+		data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L));
+		data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L));
+		data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L));
+		data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L));
+		data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L));
+		data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L));
+		data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L));
+		data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L));
+		data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L));
+		data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<Integer, Long, Integer, String, Long>> type = new TupleTypeInfo<>(
+				BasicTypeInfo.INT_TYPE_INFO,
+				BasicTypeInfo.LONG_TYPE_INFO,
+				BasicTypeInfo.INT_TYPE_INFO,
+				BasicTypeInfo.STRING_TYPE_INFO,
+				BasicTypeInfo.LONG_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple5<Integer, Long, Integer, String, Long>> getSmall5TupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
+		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
+		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
+		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<Integer, Long, Integer, String, Long>> type = new TupleTypeInfo<>(
+				BasicTypeInfo.INT_TYPE_INFO,
+				BasicTypeInfo.LONG_TYPE_INFO,
+				BasicTypeInfo.INT_TYPE_INFO,
+				BasicTypeInfo.STRING_TYPE_INFO,
+				BasicTypeInfo.LONG_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<Integer, Integer>, String>> getSmallNestedTupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple2<Tuple2<Integer, Integer>, String>> data = new ArrayList<>();
+		data.add(new Tuple2<>(new Tuple2<>(1, 1), "one"));
+		data.add(new Tuple2<>(new Tuple2<>(2, 2), "two"));
+		data.add(new Tuple2<>(new Tuple2<>(3, 3), "three"));
+
+		TupleTypeInfo<Tuple2<Tuple2<Integer, Integer>, String>> type = new TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
+				BasicTypeInfo.STRING_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<Integer, Integer>, String>> getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) {
+
+		List<Tuple2<Tuple2<Integer, Integer>, String>> data = new ArrayList<>();
+		data.add(new Tuple2<>(new Tuple2<>(1, 3), "a"));
+		data.add(new Tuple2<>(new Tuple2<>(1, 2), "a"));
+		data.add(new Tuple2<>(new Tuple2<>(2, 1), "a"));
+		data.add(new Tuple2<>(new Tuple2<>(2, 2), "b"));
+		data.add(new Tuple2<>(new Tuple2<>(3, 3), "c"));
+		data.add(new Tuple2<>(new Tuple2<>(3, 6), "c"));
+		data.add(new Tuple2<>(new Tuple2<>(4, 9), "c"));
+
+		TupleTypeInfo<Tuple2<Tuple2<Integer, Integer>, String>> type = new TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
+				BasicTypeInfo.STRING_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) {
+
+		List<Tuple3<Tuple2<Integer, Integer>, String, Integer>> data = new ArrayList<>();
+		data.add(new Tuple3<>(new Tuple2<>(1, 3), "a", 2));
+		data.add(new Tuple3<>(new Tuple2<>(1, 2), "a", 1));
+		data.add(new Tuple3<>(new Tuple2<>(2, 1), "a", 3));
+		data.add(new Tuple3<>(new Tuple2<>(2, 2), "b", 4));
+		data.add(new Tuple3<>(new Tuple2<>(3, 3), "c", 5));
+		data.add(new Tuple3<>(new Tuple2<>(3, 6), "c", 6));
+		data.add(new Tuple3<>(new Tuple2<>(4, 9), "c", 7));
+
+		TupleTypeInfo<Tuple3<Tuple2<Integer, Integer>, String, Integer>> type = new TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
+				BasicTypeInfo.STRING_TYPE_INFO,
+				BasicTypeInfo.INT_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<byte[], Integer>> getTuple2WithByteArrayDataSet(ExecutionEnvironment env) {
+		List<Tuple2<byte[], Integer>> data = new ArrayList<>();
+		data.add(new Tuple2<>(new byte[]{0, 4}, 1));
+		data.add(new Tuple2<>(new byte[]{2, 0}, 1));
+		data.add(new Tuple2<>(new byte[]{2, 0, 4}, 4));
+		data.add(new Tuple2<>(new byte[]{2, 1}, 3));
+		data.add(new Tuple2<>(new byte[]{0}, 0));
+		data.add(new Tuple2<>(new byte[]{2, 0}, 1));
+
+		TupleTypeInfo<Tuple2<byte[], Integer>> type = new TupleTypeInfo<>(
+				PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO,
+				BasicTypeInfo.INT_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<String> getStringDataSet(ExecutionEnvironment env) {
+
+		List<String> data = new ArrayList<>();
+		data.add("Hi");
+		data.add("Hello");
+		data.add("Hello world");
+		data.add("Hello world, how are you?");
+		data.add("I am fine.");
+		data.add("Luke Skywalker");
+		data.add("Random comment");
+		data.add("LOL");
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Integer> getIntegerDataSet(ExecutionEnvironment env) {
+
+		List<Integer> data = new ArrayList<>();
+		data.add(1);
+		data.add(2);
+		data.add(2);
+		data.add(3);
+		data.add(3);
+		data.add(3);
+		data.add(4);
+		data.add(4);
+		data.add(4);
+		data.add(4);
+		data.add(5);
+		data.add(5);
+		data.add(5);
+		data.add(5);
+		data.add(5);
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<CustomType> getCustomTypeDataSet(ExecutionEnvironment env) {
+
+		List<CustomType> data = new ArrayList<>();
+		data.add(new CustomType(1, 0L, "Hi"));
+		data.add(new CustomType(2, 1L, "Hello"));
+		data.add(new CustomType(2, 2L, "Hello world"));
+		data.add(new CustomType(3, 3L, "Hello world, how are you?"));
+		data.add(new CustomType(3, 4L, "I am fine."));
+		data.add(new CustomType(3, 5L, "Luke Skywalker"));
+		data.add(new CustomType(4, 6L, "Comment#1"));
+		data.add(new CustomType(4, 7L, "Comment#2"));
+		data.add(new CustomType(4, 8L, "Comment#3"));
+		data.add(new CustomType(4, 9L, "Comment#4"));
+		data.add(new CustomType(5, 10L, "Comment#5"));
+		data.add(new CustomType(5, 11L, "Comment#6"));
+		data.add(new CustomType(5, 12L, "Comment#7"));
+		data.add(new CustomType(5, 13L, "Comment#8"));
+		data.add(new CustomType(5, 14L, "Comment#9"));
+		data.add(new CustomType(6, 15L, "Comment#10"));
+		data.add(new CustomType(6, 16L, "Comment#11"));
+		data.add(new CustomType(6, 17L, "Comment#12"));
+		data.add(new CustomType(6, 18L, "Comment#13"));
+		data.add(new CustomType(6, 19L, "Comment#14"));
+		data.add(new CustomType(6, 20L, "Comment#15"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+
+	}
+
+	public static DataSet<CustomType> getSmallCustomTypeDataSet(ExecutionEnvironment env) {
+
+		List<CustomType> data = new ArrayList<>();
+		data.add(new CustomType(1, 0L, "Hi"));
+		data.add(new CustomType(2, 1L, "Hello"));
+		data.add(new CustomType(2, 2L, "Hello world"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class CustomType implements Serializable {
+
+		private static final long serialVersionUID = 1L;
+
+		public int myInt;
+		public long myLong;
+		public String myString;
+
+		public CustomType() {
+		}
+
+		public CustomType(int i, long l, String s) {
+			myInt = i;
+			myLong = l;
+			myString = s;
+		}
+
+		@Override
+		public String toString() {
+			return myInt + "," + myLong + "," + myString;
+		}
+	}
+
+	private static class CustomTypeComparator implements Comparator<CustomType> {
+		@Override
+		public int compare(CustomType o1, CustomType o2) {
+			int diff = o1.myInt - o2.myInt;
+			if (diff != 0) {
+				return diff;
+			}
+			diff = (int) (o1.myLong - o2.myLong);
+			return diff != 0 ? diff : o1.myString.compareTo(o2.myString);
+		}
+
+	}
+
+	public static DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> getSmallTuplebasedDataSet(ExecutionEnvironment env) {
+		List<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> data = new ArrayList<>();
+		data.add(new Tuple7<>(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new Tuple7<>(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new Tuple7<>(3, "Third", 30, 300, 3000L, "Three", 30000L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) {
+		List<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> data = new ArrayList<>();
+		data.add(new Tuple7<>(10000L, 10, 100, 1000L, "One", 1, "First"));
+		data.add(new Tuple7<>(20000L, 20, 200, 2000L, "Two", 2, "Second"));
+		data.add(new Tuple7<>(30000L, 30, 300, 3000L, "Three", 3, "Third"));
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getSmallPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<>();
+		data.add(new POJO(1 /*number*/, "First" /*str*/, 10 /*f0*/, 100/*f1.myInt*/, 1000L/*f1.myLong*/, "One" /*f1.myString*/, 10000L /*nestedPojo.longNumber*/));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getDuplicatePojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<>();
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<POJO> getMixedPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<>();
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x
+		data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L));
+		data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L));
+		data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L));
+		data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L));
+		data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L));
+		data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x
+		data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class POJO {
+		public int number;
+		public String str;
+		public Tuple2<Integer, CustomType> nestedTupleWithCustom;
+		public NestedPojo nestedPojo;
+		public transient Long ignoreMe;
+
+		public POJO(int i0, String s0,
+					int i1, int i2, long l0, String s1,
+					long l1) {
+			this.number = i0;
+			this.str = s0;
+			this.nestedTupleWithCustom = new Tuple2<>(i1, new CustomType(i2, l0, s1));
+			this.nestedPojo = new NestedPojo();
+			this.nestedPojo.longNumber = l1;
+		}
+
+		public POJO() {
+		}
+
+		@Override
+		public String toString() {
+			return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber;
+		}
+	}
+
+	/**
+	 * Nested POJO.
+	 */
+	public static class NestedPojo {
+		public static Object ignoreMe;
+		public long longNumber;
+
+		public NestedPojo() {
+		}
+	}
+
+	public static DataSet<CrazyNested> getCrazyNestedDataSet(ExecutionEnvironment env) {
+		List<CrazyNested> data = new ArrayList<>();
+		data.add(new CrazyNested("aa"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * Deeply nested POJO.
+	 */
+	public static class CrazyNested {
+		public CrazyNestedL1 nestLvl1;
+		public Long something; // test proper null-value handling
+
+		public CrazyNested() {
+		}
+
+		public CrazyNested(String set, String second, long s) { // additional CTor to set all fields to non-null values
+			this(set);
+			something = s;
+			nestLvl1.a = second;
+		}
+
+		public CrazyNested(String set) {
+			nestLvl1 = new CrazyNestedL1();
+			nestLvl1.nestLvl2 = new CrazyNestedL2();
+			nestLvl1.nestLvl2.nestLvl3 = new CrazyNestedL3();
+			nestLvl1.nestLvl2.nestLvl3.nestLvl4 = new CrazyNestedL4();
+			nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal = set;
+		}
+	}
+
+	/**
+	 * Nested POJO level 2.
+	 */
+	public static class CrazyNestedL1 {
+		public String a;
+		public int b;
+		public CrazyNestedL2 nestLvl2;
+	}
+
+	/**
+	 * Nested POJO level 3.
+	 */
+	public static class CrazyNestedL2 {
+		public CrazyNestedL3 nestLvl3;
+	}
+
+	/**
+	 * Nested POJO level 4.
+	 */
+	public static class CrazyNestedL3 {
+		public CrazyNestedL4 nestLvl4;
+	}
+
+	/**
+	 * Nested POJO level 5.
+	 */
+	public static class CrazyNestedL4 {
+		public String f1nal;
+	}
+
+	// Copied from TypeExtractorTest
+	private static class FromTuple extends Tuple3<String, String, Long> {
+		private static final long serialVersionUID = 1L;
+		public int special;
+	}
+
+	/**
+	 * Pojo extending from tuple WITH custom fields.
+	 */
+	public static class FromTupleWithCTor extends FromTuple {
+
+		private static final long serialVersionUID = 1L;
+
+		public FromTupleWithCTor() {}
+
+		public FromTupleWithCTor(int special, long tupleField) {
+			this.special = special;
+			this.setField(tupleField, 2);
+		}
+	}
+
+	public static DataSet<FromTupleWithCTor> getPojoExtendingFromTuple(ExecutionEnvironment env) {
+		List<FromTupleWithCTor> data = new ArrayList<>();
+		data.add(new FromTupleWithCTor(1, 10L)); // 3x
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(2, 20L)); // 2x
+		data.add(new FromTupleWithCTor(2, 20L));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO with Tuple and Writable.
+	 */
+	public static class PojoContainingTupleAndWritable {
+		public int someInt;
+		public String someString;
+		public IntWritable hadoopFan;
+		public Tuple2<Long, Long> theTuple;
+
+		public PojoContainingTupleAndWritable() {
+		}
+
+		public PojoContainingTupleAndWritable(int i, long l1, long l2) {
+			hadoopFan = new IntWritable(i);
+			someInt = i;
+			theTuple = new Tuple2<>(l1, l2);
+		}
+	}
+
+	public static DataSet<PojoContainingTupleAndWritable> getPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<PojoContainingTupleAndWritable> getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 201L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 600L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 400L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<Integer, CrazyNested, POJO>> getTupleContainingPojos(ExecutionEnvironment env) {
+		List<Tuple3<Integer, CrazyNested, POJO>> data = new ArrayList<>();
+		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x
+		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		// POJO is not initialized according to the first two fields.
+		data.add(new Tuple3<>(2, new CrazyNested("two", "duo", 2L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class Pojo1 {
+		public String a;
+		public String b;
+
+		public Pojo1() {}
+
+		public Pojo1(String a, String b) {
+			this.a = a;
+			this.b = b;
+		}
+	}
+
+	/**
+	 * Another POJO.
+	 */
+	public static class Pojo2 {
+		public String a2;
+		public String b2;
+	}
+
+	/**
+	 * Nested POJO.
+	 */
+	public static class PojoWithMultiplePojos {
+		public Pojo1 p1;
+		public Pojo2 p2;
+		public Integer i0;
+
+		public PojoWithMultiplePojos() {
+		}
+
+		public PojoWithMultiplePojos(String a, String b, String a1, String b1, Integer i0) {
+			p1 = new Pojo1();
+			p1.a = a;
+			p1.b = b;
+			p2 = new Pojo2();
+			p2.a2 = a1;
+			p2.b2 = b1;
+			this.i0 = i0;
+		}
+	}
+
+	public static DataSet<PojoWithMultiplePojos> getPojoWithMultiplePojos(ExecutionEnvironment env) {
+		List<PojoWithMultiplePojos> data = new ArrayList<>();
+		data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * Custom enum.
+	 */
+	public enum Category {
+		CAT_A, CAT_B
+	}
+
+	/**
+	 * POJO with Date and enum.
+	 */
+	public static class PojoWithDateAndEnum {
+		public String group;
+		public Date date;
+		public Category cat;
+	}
+
+	public static DataSet<PojoWithDateAndEnum> getPojoWithDateAndEnum(ExecutionEnvironment env) {
+		List<PojoWithDateAndEnum> data = new ArrayList<>();
+
+		PojoWithDateAndEnum one = new PojoWithDateAndEnum();
+		one.group = "a"; one.date = new Date(666); one.cat = Category.CAT_A;
+		data.add(one);
+
+		PojoWithDateAndEnum two = new PojoWithDateAndEnum();
+		two.group = "a"; two.date = new Date(666); two.cat = Category.CAT_A;
+		data.add(two);
+
+		PojoWithDateAndEnum three = new PojoWithDateAndEnum();
+		three.group = "b"; three.date = new Date(666); three.cat = Category.CAT_B;
+		data.add(three);
+
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO with collection.
+	 */
+	public static class PojoWithCollection {
+		public List<Pojo1> pojos;
+		public int key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	/**
+	 * POJO with generic collection.
+	 */
+	public static class PojoWithCollectionGeneric {
+		public List<Pojo1> pojos;
+		public int key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+		private PojoWithDateAndEnum makeMeGeneric;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	public static DataSet<PojoWithCollection> getPojoWithCollection(ExecutionEnvironment env) {
+		List<PojoWithCollection> data = new ArrayList<>();
+
+		List<Pojo1> pojosList1 = new ArrayList<>();
+		pojosList1.add(new Pojo1("a", "aa"));
+		pojosList1.add(new Pojo1("b", "bb"));
+
+		List<Pojo1> pojosList2 = new ArrayList<>();
+		pojosList2.add(new Pojo1("a2", "aa2"));
+		pojosList2.add(new Pojo1("b2", "bb2"));
+
+		PojoWithCollection pwc1 = new PojoWithCollection();
+		pwc1.pojos = pojosList1;
+		pwc1.key = 0;
+		pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc1.scalaBigInt = BigInt.int2bigInt(10);
+		pwc1.bigDecimalKeepItNull = null;
+
+		// use calendar to make it stable across time zones
+		GregorianCalendar gcl1 = new GregorianCalendar(2033, 4, 18);
+		pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis());
+		pwc1.mixed = new ArrayList<>();
+		Map<String, Integer> map = new HashMap<>();
+		map.put("someKey", 1); // map.put("anotherKey", 2); map.put("third", 3);
+		pwc1.mixed.add(map);
+		pwc1.mixed.add(new File("/this/is/wrong"));
+		pwc1.mixed.add("uhlala");
+
+		PojoWithCollection pwc2 = new PojoWithCollection();
+		pwc2.pojos = pojosList2;
+		pwc2.key = 0;
+		pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc2.scalaBigInt = BigInt.int2bigInt(31104000);
+		pwc2.bigDecimalKeepItNull = null;
+
+		GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3);
+		pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976
+
+		data.add(pwc1);
+		data.add(pwc2);
+
+		return env.fromCollection(data);
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java
new file mode 100644
index 0000000..0262617
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/util/ValueCollectionDataSets.java
@@ -0,0 +1,775 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators.util;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.ValueTypeInfo;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.StringValue;
+
+import org.apache.hadoop.io.IntWritable;
+
+import java.io.File;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import scala.math.BigInt;
+
+/**
+ * #######################################################################################################
+ * 			BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA.
+ * 			IF YOU MODIFY THE DATA MAKE SURE YOU CHECK THAT ALL TESTS ARE STILL WORKING!
+ * #######################################################################################################
+ */
+public class ValueCollectionDataSets {
+
+	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> get3TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new IntValue(1), new LongValue(1L), new StringValue("Hi")));
+		data.add(new Tuple3<>(new IntValue(2), new LongValue(2L), new StringValue("Hello")));
+		data.add(new Tuple3<>(new IntValue(3), new LongValue(2L), new StringValue("Hello world")));
+		data.add(new Tuple3<>(new IntValue(4), new LongValue(3L), new StringValue("Hello world, how are you?")));
+		data.add(new Tuple3<>(new IntValue(5), new LongValue(3L), new StringValue("I am fine.")));
+		data.add(new Tuple3<>(new IntValue(6), new LongValue(3L), new StringValue("Luke Skywalker")));
+		data.add(new Tuple3<>(new IntValue(7), new LongValue(4L), new StringValue("Comment#1")));
+		data.add(new Tuple3<>(new IntValue(8), new LongValue(4L), new StringValue("Comment#2")));
+		data.add(new Tuple3<>(new IntValue(9), new LongValue(4L), new StringValue("Comment#3")));
+		data.add(new Tuple3<>(new IntValue(10), new LongValue(4L), new StringValue("Comment#4")));
+		data.add(new Tuple3<>(new IntValue(11), new LongValue(5L), new StringValue("Comment#5")));
+		data.add(new Tuple3<>(new IntValue(12), new LongValue(5L), new StringValue("Comment#6")));
+		data.add(new Tuple3<>(new IntValue(13), new LongValue(5L), new StringValue("Comment#7")));
+		data.add(new Tuple3<>(new IntValue(14), new LongValue(5L), new StringValue("Comment#8")));
+		data.add(new Tuple3<>(new IntValue(15), new LongValue(5L), new StringValue("Comment#9")));
+		data.add(new Tuple3<>(new IntValue(16), new LongValue(6L), new StringValue("Comment#10")));
+		data.add(new Tuple3<>(new IntValue(17), new LongValue(6L), new StringValue("Comment#11")));
+		data.add(new Tuple3<>(new IntValue(18), new LongValue(6L), new StringValue("Comment#12")));
+		data.add(new Tuple3<>(new IntValue(19), new LongValue(6L), new StringValue("Comment#13")));
+		data.add(new Tuple3<>(new IntValue(20), new LongValue(6L), new StringValue("Comment#14")));
+		data.add(new Tuple3<>(new IntValue(21), new LongValue(6L), new StringValue("Comment#15")));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> getSmall3TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new IntValue(1), new LongValue(1L), new StringValue("Hi")));
+		data.add(new Tuple3<>(new IntValue(2), new LongValue(2L), new StringValue("Hello")));
+		data.add(new Tuple3<>(new IntValue(3), new LongValue(2L), new StringValue("Hello world")));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> get5TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
+
+		data.add(new Tuple5<>(new IntValue(1), new LongValue(1L), new IntValue(0), new StringValue("Hallo"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(2L), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(3L), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(4L), new IntValue(3), new StringValue("Hallo Welt wie gehts?"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(5L), new IntValue(4), new StringValue("ABC"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(3), new LongValue(6L), new IntValue(5), new StringValue("BCD"), new LongValue(3L)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(7L), new IntValue(6), new StringValue("CDE"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(8L), new IntValue(7), new StringValue("DEF"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(9L), new IntValue(8), new StringValue("EFG"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(4), new LongValue(10L), new IntValue(9), new StringValue("FGH"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(11L), new IntValue(10), new StringValue("GHI"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(12L), new IntValue(11), new StringValue("HIJ"), new LongValue(3L)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(13L), new IntValue(12), new StringValue("IJK"), new LongValue(3L)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(14L), new IntValue(13), new StringValue("JKL"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(5), new LongValue(15L), new IntValue(14), new StringValue("KLM"), new LongValue(2L)));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
+			TupleTypeInfo<>(
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> getSmall5TupleDataSet(ExecutionEnvironment env) {
+		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
+
+		data.add(new Tuple5<>(new IntValue(1), new LongValue(1L), new IntValue(0), new StringValue("Hallo"), new LongValue(1L)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(2L), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2L)));
+		data.add(new Tuple5<>(new IntValue(2), new LongValue(3L), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1L)));
+
+		Collections.shuffle(data);
+
+		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
+			TupleTypeInfo<>(
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO,
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.LONG_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getSmallNestedTupleDataSet(ExecutionEnvironment env) {
+		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(1)), new StringValue("one")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("two")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("three")));
+
+		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) {
+		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(3)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(2)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(1)), new StringValue("a")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("b")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("c")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(6)), new StringValue("c")));
+		data.add(new Tuple2<>(new Tuple2<>(new IntValue(4), new IntValue(9)), new StringValue("c")));
+
+		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) {
+		List<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> data = new ArrayList<>();
+
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(3)), new StringValue("a"), new IntValue(2)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(2)), new StringValue("a"), new IntValue(1)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(1)), new StringValue("a"), new IntValue(3)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(2)), new StringValue("b"), new IntValue(4)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(3)), new StringValue("c"), new IntValue(5)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(6)), new StringValue("c"), new IntValue(6)));
+		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(4), new IntValue(9)), new StringValue("c"), new IntValue(7)));
+
+		TupleTypeInfo<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> type = new
+			TupleTypeInfo<>(
+				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
+				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
+				ValueTypeInfo.INT_VALUE_TYPE_INFO
+		);
+
+		return env.fromCollection(data, type);
+	}
+
+	public static DataSet<StringValue> getStringDataSet(ExecutionEnvironment env) {
+		List<StringValue> data = new ArrayList<>();
+
+		data.add(new StringValue("Hi"));
+		data.add(new StringValue("Hello"));
+		data.add(new StringValue("Hello world"));
+		data.add(new StringValue("Hello world, how are you?"));
+		data.add(new StringValue("I am fine."));
+		data.add(new StringValue("Luke Skywalker"));
+		data.add(new StringValue("Random comment"));
+		data.add(new StringValue("LOL"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<IntValue> getIntDataSet(ExecutionEnvironment env) {
+		List<IntValue> data = new ArrayList<>();
+
+		data.add(new IntValue(1));
+		data.add(new IntValue(2));
+		data.add(new IntValue(2));
+		data.add(new IntValue(3));
+		data.add(new IntValue(3));
+		data.add(new IntValue(3));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(4));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+		data.add(new IntValue(5));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<CustomType> getCustomTypeDataSet(ExecutionEnvironment env) {
+		List<CustomType> data = new ArrayList<CustomType>();
+
+		data.add(new CustomType(1, 0L, "Hi"));
+		data.add(new CustomType(2, 1L, "Hello"));
+		data.add(new CustomType(2, 2L, "Hello world"));
+		data.add(new CustomType(3, 3L, "Hello world, how are you?"));
+		data.add(new CustomType(3, 4L, "I am fine."));
+		data.add(new CustomType(3, 5L, "Luke Skywalker"));
+		data.add(new CustomType(4, 6L, "Comment#1"));
+		data.add(new CustomType(4, 7L, "Comment#2"));
+		data.add(new CustomType(4, 8L, "Comment#3"));
+		data.add(new CustomType(4, 9L, "Comment#4"));
+		data.add(new CustomType(5, 10L, "Comment#5"));
+		data.add(new CustomType(5, 11L, "Comment#6"));
+		data.add(new CustomType(5, 12L, "Comment#7"));
+		data.add(new CustomType(5, 13L, "Comment#8"));
+		data.add(new CustomType(5, 14L, "Comment#9"));
+		data.add(new CustomType(6, 15L, "Comment#10"));
+		data.add(new CustomType(6, 16L, "Comment#11"));
+		data.add(new CustomType(6, 17L, "Comment#12"));
+		data.add(new CustomType(6, 18L, "Comment#13"));
+		data.add(new CustomType(6, 19L, "Comment#14"));
+		data.add(new CustomType(6, 20L, "Comment#15"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<CustomType> getSmallCustomTypeDataSet(ExecutionEnvironment env) {
+		List<CustomType> data = new ArrayList<CustomType>();
+
+		data.add(new CustomType(1, 0L, "Hi"));
+		data.add(new CustomType(2, 1L, "Hello"));
+		data.add(new CustomType(2, 2L, "Hello world"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class CustomType implements Serializable {
+
+		private static final long serialVersionUID = 1L;
+
+		public IntValue myInt;
+		public LongValue myLong;
+		public StringValue myString;
+
+		public CustomType() {
+		}
+
+		public CustomType(int i, long l, String s) {
+			myInt = new IntValue(i);
+			myLong = new LongValue(l);
+			myString = new StringValue(s);
+		}
+
+		@Override
+		public String toString() {
+			return myInt + "," + myLong + "," + myString;
+		}
+	}
+
+	private static class CustomTypeComparator implements Comparator<CustomType> {
+
+		@Override
+		public int compare(CustomType o1, CustomType o2) {
+			int diff = o1.myInt.getValue() - o2.myInt.getValue();
+			if (diff != 0) {
+				return diff;
+			}
+			diff = (int) (o1.myLong.getValue() - o2.myLong.getValue());
+			return diff != 0 ? diff : o1.myString.getValue().compareTo(o2.myString.getValue());
+		}
+
+	}
+
+	private static DataSet<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> getSmallTuplebasedDataSet(ExecutionEnvironment env) {
+		List<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> data = new ArrayList<>();
+
+		data.add(new Tuple7<>(new IntValue(1), new StringValue("First"), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new LongValue(10000L)));
+		data.add(new Tuple7<>(new IntValue(2), new StringValue("Second"), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new LongValue(20000L)));
+		data.add(new Tuple7<>(new IntValue(3), new StringValue("Third"), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new LongValue(30000L)));
+
+		return env.fromCollection(data);
+	}
+
+	private static DataSet<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) {
+		List<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> data = new ArrayList<>();
+
+		data.add(new Tuple7<>(new LongValue(10000L), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new IntValue(1), new StringValue("First")));
+		data.add(new Tuple7<>(new LongValue(20000L), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new IntValue(2), new StringValue("Second")));
+		data.add(new Tuple7<>(new LongValue(30000L), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new IntValue(3), new StringValue("Third")));
+
+		return env.fromCollection(data);
+	}
+
+	private static DataSet<POJO> getSmallPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1 /*number*/, "First" /*str*/, 10 /*f0*/, 100/*f1.myInt*/, 1000L/*f1.myLong*/, "One" /*f1.myString*/, 10000L /*nestedPojo.longNumber*/));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+
+		return env.fromCollection(data);
+	}
+
+	private static DataSet<POJO> getDuplicatePojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
+		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x
+		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
+
+		return env.fromCollection(data);
+	}
+
+	private static DataSet<POJO> getMixedPojoDataSet(ExecutionEnvironment env) {
+		List<POJO> data = new ArrayList<POJO>();
+
+		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x
+		data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L));
+		data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L));
+		data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L));
+		data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L));
+		data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L));
+		data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x
+		data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L));
+
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class POJO {
+		public IntValue number;
+		public StringValue str;
+		public Tuple2<IntValue, CustomType> nestedTupleWithCustom;
+		public NestedPojo nestedPojo;
+		public transient LongValue ignoreMe;
+
+		public POJO(int i0, String s0,
+					int i1, int i2, long l0, String s1,
+					long l1) {
+			this.number = new IntValue(i0);
+			this.str = new StringValue(s0);
+			this.nestedTupleWithCustom = new Tuple2<>(new IntValue(i1), new CustomType(i2, l0, s1));
+			this.nestedPojo = new NestedPojo();
+			this.nestedPojo.longNumber = new LongValue(l1);
+		}
+
+		public POJO() {
+		}
+
+		@Override
+		public String toString() {
+			return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber;
+		}
+	}
+
+	/**
+	 * Nested POJO.
+	 */
+	public static class NestedPojo {
+		public static Object ignoreMe;
+		public LongValue longNumber;
+
+		public NestedPojo() {
+		}
+	}
+
+	private static DataSet<CrazyNested> getCrazyNestedDataSet(ExecutionEnvironment env) {
+		List<CrazyNested> data = new ArrayList<CrazyNested>();
+
+		data.add(new CrazyNested("aa"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("bb"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+		data.add(new CrazyNested("cc"));
+
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * Deeply nested POJO.
+	 */
+	public static class CrazyNested {
+		public CrazyNestedL1 nestLvl1;
+		public LongValue something; // test proper null-value handling
+
+		public CrazyNested() {
+		}
+
+		public CrazyNested(String set, String second, long s) { // additional CTor to set all fields to non-null values
+			this(set);
+			something = new LongValue(s);
+			nestLvl1.a = new StringValue(second);
+		}
+
+		public CrazyNested(String set) {
+			nestLvl1 = new CrazyNestedL1();
+			nestLvl1.nestLvl2 = new CrazyNestedL2();
+			nestLvl1.nestLvl2.nestLvl3 = new CrazyNestedL3();
+			nestLvl1.nestLvl2.nestLvl3.nestLvl4 = new CrazyNestedL4();
+			nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal = new StringValue(set);
+		}
+	}
+
+	/**
+	 * Nested POJO level 2.
+	 */
+	public static class CrazyNestedL1 {
+		public StringValue a;
+		public IntValue b;
+		public CrazyNestedL2 nestLvl2;
+	}
+
+	/**
+	 * Nested POJO level 3.
+	 */
+	public static class CrazyNestedL2 {
+		public CrazyNestedL3 nestLvl3;
+	}
+
+	/**
+	 * Nested POJO level 4.
+	 */
+	public static class CrazyNestedL3 {
+		public CrazyNestedL4 nestLvl4;
+	}
+
+	/**
+	 * Nested POJO level 5.
+	 */
+	public static class CrazyNestedL4 {
+		public StringValue f1nal;
+	}
+
+	// Copied from TypeExtractorTest
+	private static class FromTuple extends Tuple3<StringValue, StringValue, LongValue> {
+		private static final long serialVersionUID = 1L;
+		public IntValue special;
+	}
+
+	private static class FromTupleWithCTor extends FromTuple {
+
+		private static final long serialVersionUID = 1L;
+
+		public FromTupleWithCTor() {}
+
+		public FromTupleWithCTor(int special, long tupleField) {
+			this.special = new IntValue(special);
+			this.setField(new LongValue(tupleField), 2);
+		}
+	}
+
+	public static DataSet<FromTupleWithCTor> getPojoExtendingFromTuple(ExecutionEnvironment env) {
+		List<FromTupleWithCTor> data = new ArrayList<>();
+		data.add(new FromTupleWithCTor(1, 10L)); // 3x
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(1, 10L));
+		data.add(new FromTupleWithCTor(2, 20L)); // 2x
+		data.add(new FromTupleWithCTor(2, 20L));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO with Tuple and Writable.
+	 */
+	public static class PojoContainingTupleAndWritable {
+		public IntValue someInt;
+		public StringValue someString;
+		public IntWritable hadoopFan;
+		public Tuple2<LongValue, LongValue> theTuple;
+
+		public PojoContainingTupleAndWritable() {
+		}
+
+		public PojoContainingTupleAndWritable(int i, long l1, long l2) {
+			hadoopFan = new IntWritable(i);
+			someInt = new IntValue(i);
+			theTuple = new Tuple2<>(new LongValue(l1), new LongValue(l2));
+		}
+	}
+
+	public static DataSet<PojoContainingTupleAndWritable> getPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<PojoContainingTupleAndWritable> getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) {
+		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
+		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
+		data.add(new PojoContainingTupleAndWritable(2, 20L, 201L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 200L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 600L));
+		data.add(new PojoContainingTupleAndWritable(2, 30L, 400L));
+		return env.fromCollection(data);
+	}
+
+	public static DataSet<Tuple3<IntValue, CrazyNested, POJO>> getTupleContainingPojos(ExecutionEnvironment env) {
+		List<Tuple3<IntValue, CrazyNested, POJO>> data = new ArrayList<>();
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
+		// POJO is not initialized according to the first two fields.
+		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(2), new CrazyNested("two", "duo", 2L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class Pojo1 {
+		public StringValue a;
+		public StringValue b;
+
+		public Pojo1() {}
+
+		public Pojo1(String a, String b) {
+			this.a = new StringValue(a);
+			this.b = new StringValue(b);
+		}
+	}
+
+	/**
+	 * POJO.
+	 */
+	public static class Pojo2 {
+		public StringValue a2;
+		public StringValue b2;
+	}
+
+	/**
+	 * Nested POJO.
+	 */
+	public static class PojoWithMultiplePojos {
+		public Pojo1 p1;
+		public Pojo2 p2;
+		public IntValue i0;
+
+		public PojoWithMultiplePojos() {
+		}
+
+		public PojoWithMultiplePojos(String a, String b, String a1, String b1, int i0) {
+			p1 = new Pojo1();
+			p1.a = new StringValue(a);
+			p1.b = new StringValue(b);
+			p2 = new Pojo2();
+			p2.a2 = new StringValue(a1);
+			p2.b2 = new StringValue(b1);
+			this.i0 = new IntValue(i0);
+		}
+	}
+
+	public static DataSet<PojoWithMultiplePojos> getPojoWithMultiplePojos(ExecutionEnvironment env) {
+		List<PojoWithMultiplePojos> data = new ArrayList<>();
+		data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * Custom enum.
+	 */
+	public enum Category {
+		CAT_A, CAT_B;
+	}
+
+	/**
+	 * POJO with Data and enum.
+	 */
+	public static class PojoWithDateAndEnum {
+		public StringValue group;
+		public Date date;
+		public Category cat;
+	}
+
+	public static DataSet<PojoWithDateAndEnum> getPojoWithDateAndEnum(ExecutionEnvironment env) {
+		List<PojoWithDateAndEnum> data = new ArrayList<PojoWithDateAndEnum>();
+
+		PojoWithDateAndEnum one = new PojoWithDateAndEnum();
+		one.group = new StringValue("a");
+		one.date = new Date(666);
+		one.cat = Category.CAT_A;
+		data.add(one);
+
+		PojoWithDateAndEnum two = new PojoWithDateAndEnum();
+		two.group = new StringValue("a");
+		two.date = new Date(666);
+		two.cat = Category.CAT_A;
+		data.add(two);
+
+		PojoWithDateAndEnum three = new PojoWithDateAndEnum();
+		three.group = new StringValue("b");
+		three.date = new Date(666);
+		three.cat = Category.CAT_B;
+		data.add(three);
+
+		return env.fromCollection(data);
+	}
+
+	/**
+	 * POJO with collection.
+	 */
+	public static class PojoWithCollection {
+		public List<Pojo1> pojos;
+		public IntValue key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	/**
+	 * POJO with generic collection.
+	 */
+	public static class PojoWithCollectionGeneric {
+		public List<Pojo1> pojos;
+		public IntValue key;
+		public java.sql.Date sqlDate;
+		public BigInteger bigInt;
+		public BigDecimal bigDecimalKeepItNull;
+		public BigInt scalaBigInt;
+		public List<Object> mixed;
+		private PojoWithDateAndEnum makeMeGeneric;
+
+		@Override
+		public String toString() {
+			return "PojoWithCollection{" +
+					"pojos.size()=" + pojos.size() +
+					", key=" + key +
+					", sqlDate=" + sqlDate +
+					", bigInt=" + bigInt +
+					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
+					", scalaBigInt=" + scalaBigInt +
+					", mixed=" + mixed +
+					'}';
+		}
+	}
+
+	public static DataSet<PojoWithCollection> getPojoWithCollection(ExecutionEnvironment env) {
+		List<PojoWithCollection> data = new ArrayList<>();
+
+		List<Pojo1> pojosList1 = new ArrayList<>();
+		pojosList1.add(new Pojo1("a", "aa"));
+		pojosList1.add(new Pojo1("b", "bb"));
+
+		List<Pojo1> pojosList2 = new ArrayList<>();
+		pojosList2.add(new Pojo1("a2", "aa2"));
+		pojosList2.add(new Pojo1("b2", "bb2"));
+
+		PojoWithCollection pwc1 = new PojoWithCollection();
+		pwc1.pojos = pojosList1;
+		pwc1.key = new IntValue(0);
+		pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc1.scalaBigInt = BigInt.int2bigInt(10);
+		pwc1.bigDecimalKeepItNull = null;
+
+		// use calendar to make it stable across time zones
+		GregorianCalendar gcl1 = new GregorianCalendar(2033, 04, 18);
+		pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis());
+		pwc1.mixed = new ArrayList<Object>();
+		Map<StringValue, IntValue> map = new HashMap<>();
+		map.put(new StringValue("someKey"), new IntValue(1));
+		pwc1.mixed.add(map);
+		pwc1.mixed.add(new File("/this/is/wrong"));
+		pwc1.mixed.add("uhlala");
+
+		PojoWithCollection pwc2 = new PojoWithCollection();
+		pwc2.pojos = pojosList2;
+		pwc2.key = new IntValue(0);
+		pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
+		pwc2.scalaBigInt = BigInt.int2bigInt(31104000);
+		pwc2.bigDecimalKeepItNull = null;
+
+		GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3);
+		pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976
+
+		data.add(pwc1);
+		data.add(pwc2);
+
+		return env.fromCollection(data);
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java
index 32fe7c5..a13f523 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.optimizer.examples;
 
-import static org.junit.Assert.*;
-
-import java.util.Arrays;
-import java.util.Collection;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.functions.GroupCombineFunction;
@@ -41,29 +36,39 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.OperatorResolver;
 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.optimizer.util.CompilerTestBase;
-import org.apache.flink.optimizer.util.OperatorResolver;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Validate the compilation and result of a single iteration of KMeans.
+ */
 @SuppressWarnings("serial")
 public class KMeansSingleStepTest extends CompilerTestBase {
-	
+
 	private static final String DATAPOINTS = "Data Points";
 	private static final String CENTERS = "Centers";
-	
+
 	private static final String MAPPER_NAME = "Find Nearest Centers";
 	private static final String REDUCER_NAME = "Recompute Center Positions";
-	
+
 	private static final String SINK = "New Center Positions";
-	
+
 	private final FieldList set0 = new FieldList(0);
-	
-	
+
 	@Test
 	public void testCompileKMeansSingleStepWithStats() {
 
@@ -71,11 +76,11 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		p.setExecutionConfig(new ExecutionConfig());
 		// set the statistics
 		OperatorResolver cr = getContractResolver(p);
-		GenericDataSourceBase<?,?> pointsSource = cr.getNode(DATAPOINTS);
-		GenericDataSourceBase<?,?> centersSource = cr.getNode(CENTERS);
+		GenericDataSourceBase<?, ?> pointsSource = cr.getNode(DATAPOINTS);
+		GenericDataSourceBase<?, ?> centersSource = cr.getNode(CENTERS);
 		setSourceStatistics(pointsSource, 100L * 1024 * 1024 * 1024, 32f);
 		setSourceStatistics(centersSource, 1024 * 1024, 32f);
-		
+
 		OptimizedPlan plan = compileWithStats(p);
 		checkPlan(plan);
 	}
@@ -88,33 +93,31 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		OptimizedPlan plan = compileNoStats(p);
 		checkPlan(plan);
 	}
-	
-	
+
 	private void checkPlan(OptimizedPlan plan) {
-		
+
 		OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan);
-		
+
 		final SinkPlanNode sink = or.getNode(SINK);
 		final SingleInputPlanNode reducer = or.getNode(REDUCER_NAME);
 		final SingleInputPlanNode combiner = (SingleInputPlanNode) reducer.getPredecessor();
 		final SingleInputPlanNode mapper = or.getNode(MAPPER_NAME);
-		
+
 		// check the mapper
 		assertEquals(1, mapper.getBroadcastInputs().size());
 		assertEquals(ShipStrategyType.FORWARD, mapper.getInput().getShipStrategy());
 		assertEquals(ShipStrategyType.BROADCAST, mapper.getBroadcastInputs().get(0).getShipStrategy());
-		
+
 		assertEquals(LocalStrategy.NONE, mapper.getInput().getLocalStrategy());
 		assertEquals(LocalStrategy.NONE, mapper.getBroadcastInputs().get(0).getLocalStrategy());
-		
+
 		assertEquals(DriverStrategy.MAP, mapper.getDriverStrategy());
-		
+
 		assertNull(mapper.getInput().getLocalStrategyKeys());
 		assertNull(mapper.getInput().getLocalStrategySortOrder());
 		assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategyKeys());
 		assertNull(mapper.getBroadcastInputs().get(0).getLocalStrategySortOrder());
-		
-		
+
 		// check the combiner
 		Assert.assertNotNull(combiner);
 		assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
@@ -124,7 +127,7 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		assertNull(combiner.getInput().getLocalStrategySortOrder());
 		assertEquals(set0, combiner.getKeys(0));
 		assertEquals(set0, combiner.getKeys(1));
-		
+
 		// check the reducer
 		assertEquals(ShipStrategyType.PARTITION_HASH, reducer.getInput().getShipStrategy());
 		assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy());
@@ -132,7 +135,7 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		assertEquals(set0, reducer.getKeys(0));
 		assertEquals(set0, reducer.getInput().getLocalStrategyKeys());
 		assertTrue(Arrays.equals(reducer.getInput().getLocalStrategySortOrder(), reducer.getSortOrders(0)));
-		
+
 		// check the sink
 		assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
 		assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy());
@@ -143,7 +146,7 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		PreviewPlanEnvironment env = new PreviewPlanEnvironment();
 		env.setAsContext();
 		try {
-			KMeans(new String[]{IN_FILE, IN_FILE, OUT_FILE, "20"});
+			kmeans(new String[]{IN_FILE, IN_FILE, OUT_FILE, "20"});
 		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
@@ -153,7 +156,7 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		return env.getPlan();
 	}
 
-	public static void KMeans(String[] args) throws Exception {
+	public static void kmeans(String[] args) throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
 		DataSet<Point> points = env.readCsvFile(args[0])
@@ -183,14 +186,17 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		DataSet<Tuple3<Integer, Point, Integer>> newCentroids = points
 				.map(new SelectNearestCenter()).name(MAPPER_NAME).withBroadcastSet(centroids, "centroids");
 
-		DataSet<Tuple3<Integer, Point, Integer>> recomputeClusterCenter
-				= newCentroids.groupBy(0).reduceGroup(new RecomputeClusterCenter()).name(REDUCER_NAME);
+		DataSet<Tuple3<Integer, Point, Integer>> recomputeClusterCenter =
+				newCentroids.groupBy(0).reduceGroup(new RecomputeClusterCenter()).name(REDUCER_NAME);
 
 		recomputeClusterCenter.project(0, 1).writeAsCsv(args[2], "\n", " ").name(SINK);
 
 		env.execute("KMeans Example");
 	}
 
+	/**
+	 * Two-dimensional point.
+	 */
 	public static class Point extends Tuple2<Double, Double> {
 		public Point(double x, double y) {
 			this.f0 = x;
@@ -218,6 +224,9 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		}
 	}
 
+	/**
+	 * Center of a cluster.
+	 */
 	public static class Centroid extends Tuple2<Integer, Point> {
 
 		public Centroid(int id, double x, double y) {
@@ -234,7 +243,7 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 	/**
 	 * Determines the closest cluster center for a data point.
 	 */
-	public static final class SelectNearestCenter extends RichMapFunction<Point, Tuple3<Integer, Point, Integer>> {
+	private static final class SelectNearestCenter extends RichMapFunction<Point, Tuple3<Integer, Point, Integer>> {
 		private Collection<Centroid> centroids;
 
 		@Override
@@ -257,10 +266,9 @@ public class KMeansSingleStepTest extends CompilerTestBase {
 		}
 	}
 
-	public static final class RecomputeClusterCenter implements
+	private static final class RecomputeClusterCenter implements
 		GroupReduceFunction<Tuple3<Integer, Point, Integer>, Tuple3<Integer, Point, Integer>>,
-		GroupCombineFunction<Tuple3<Integer, Point, Integer>, Tuple3<Integer, Point, Integer>>
-	{
+		GroupCombineFunction<Tuple3<Integer, Point, Integer>, Tuple3<Integer, Point, Integer>> {
 
 		@Override
 		public void reduce(Iterable<Tuple3<Integer, Point, Integer>> values, Collector<Tuple3<Integer, Point, Integer>> out) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java
index 80e3ae0..caccd81 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/RelationalQueryCompilerTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.optimizer.examples;
 
-import java.util.Arrays;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
@@ -43,35 +41,37 @@ import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.optimizer.util.OperatorResolver;
 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.optimizer.util.CompilerTestBase;
-import org.apache.flink.optimizer.util.OperatorResolver;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Arrays;
+
 /**
  * Tests TPCH Q3 (simplified) under various input conditions.
  */
 @SuppressWarnings("serial")
 public class RelationalQueryCompilerTest extends CompilerTestBase {
-	
+
 	private static final String ORDERS = "Orders";
 	private static final String LINEITEM = "LineItems";
 	private static final String MAPPER_NAME = "FilterO";
 	private static final String JOIN_NAME = "JoinLiO";
 	private static final String REDUCE_NAME = "AggLiO";
 	private static final String SINK = "Output";
-	
+
 	private final FieldList set0 = new FieldList(0);
-	private final FieldList set01 = new FieldList(0,1);
+	private final FieldList set01 = new FieldList(0, 1);
 	private final ExecutionConfig defaultExecutionConfig = new ExecutionConfig();
-	
+
 	// ------------------------------------------------------------------------
-	
-	
+
 	/**
 	 * Verifies that a robust repartitioning plan with a hash join is created in the absence of statistics.
 	 */
@@ -82,9 +82,9 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			p.setExecutionConfig(defaultExecutionConfig);
 			// compile
 			final OptimizedPlan plan = compileNoStats(p);
-			
+
 			final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan);
-			
+
 			// get the nodes from the final plan
 			final SinkPlanNode sink = or.getNode(SINK);
 			final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME);
@@ -92,7 +92,7 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 					(SingleInputPlanNode) reducer.getPredecessor() : null;
 			final DualInputPlanNode join = or.getNode(JOIN_NAME);
 			final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME);
-			
+
 			// verify the optimizer choices
 			checkStandardStrategies(filteringMapper, join, combiner, reducer, sink);
 			Assert.assertTrue(checkRepartitionShipStrategies(join, reducer, combiner));
@@ -102,16 +102,16 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			Assert.fail(e.getMessage());
 		}
 	}
-	
+
 	/**
 	 * Checks if any valid plan is produced. Hash joins are expected to build the orders side, as the statistics
 	 * indicate this to be the smaller one.
 	 */
 	@Test
 	public void testQueryAnyValidPlan() {
-		testQueryGeneric(1024*1024*1024L, 8*1024*1024*1024L, 0.05f, 0.05f, true, true, true, false, true);
+		testQueryGeneric(1024 * 1024 * 1024L, 8 * 1024 * 1024 * 1024L, 0.05f, 0.05f, true, true, true, false, true);
 	}
-	
+
 	/**
 	 * Verifies that the plan compiles in the presence of empty size=0 estimates.
 	 */
@@ -119,23 +119,23 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 	public void testQueryWithSizeZeroInputs() {
 		testQueryGeneric(0, 0, 0.1f, 0.5f, true, true, true, false, true);
 	}
-	
+
 	/**
 	 * Statistics that push towards a broadcast join.
 	 */
 	@Test
 	public void testQueryWithStatsForBroadcastHash() {
-		testQueryGeneric(1024L*1024*1024*1024, 1024L*1024*1024*1024, 0.01f, 0.05f, true, false, true, false, false);
+		testQueryGeneric(1024L * 1024 * 1024 * 1024, 1024L * 1024 * 1024 * 1024, 0.01f, 0.05f, true, false, true, false, false);
 	}
-	
+
 	/**
 	 * Statistics that push towards a broadcast join.
 	 */
 	@Test
 	public void testQueryWithStatsForRepartitionAny() {
-		testQueryGeneric(100L*1024*1024*1024*1024, 100L*1024*1024*1024*1024, 0.1f, 0.5f, false, true, true, true, true);
+		testQueryGeneric(100L * 1024 * 1024 * 1024 * 1024, 100L * 1024 * 1024 * 1024 * 1024, 0.1f, 0.5f, false, true, true, true, true);
 	}
-	
+
 	/**
 	 * Statistics that push towards a repartition merge join. If the join blows the data volume up significantly,
 	 * re-exploiting the sorted order is cheaper.
@@ -146,45 +146,43 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 		p.setExecutionConfig(defaultExecutionConfig);
 		// set compiler hints
 		OperatorResolver cr = getContractResolver(p);
-		DualInputOperator<?,?,?,?> match = cr.getNode(JOIN_NAME);
+		DualInputOperator<?, ?, ?, ?> match = cr.getNode(JOIN_NAME);
 		match.getCompilerHints().setFilterFactor(100f);
-		
-		testQueryGeneric(100L*1024*1024*1024*1024, 100L*1024*1024*1024*1024, 0.01f, 100f, false, true, false, false, true);
+
+		testQueryGeneric(100L * 1024 * 1024 * 1024 * 1024, 100L * 1024 * 1024 * 1024 * 1024, 0.01f, 100f, false, true, false, false, true);
 	}
-	
+
 	// ------------------------------------------------------------------------
-	private void testQueryGeneric(long orderSize, long lineItemSize, 
+	private void testQueryGeneric(long orderSize, long lineItemSize,
 			float ordersFilterFactor, float joinFilterFactor,
 			boolean broadcastOkay, boolean partitionedOkay,
-			boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay)
-	{
+			boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay) {
 		Plan p = getTPCH3Plan();
 		p.setExecutionConfig(defaultExecutionConfig);
 		testQueryGeneric(p, orderSize, lineItemSize, ordersFilterFactor, joinFilterFactor, broadcastOkay, partitionedOkay, hashJoinFirstOkay, hashJoinSecondOkay, mergeJoinOkay);
 	}
-		
-	private void testQueryGeneric(Plan p, long orderSize, long lineitemSize, 
-			float orderSelectivity, float joinSelectivity, 
+
+	private void testQueryGeneric(Plan p, long orderSize, long lineitemSize,
+			float orderSelectivity, float joinSelectivity,
 			boolean broadcastOkay, boolean partitionedOkay,
-			boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay)
-	{
+			boolean hashJoinFirstOkay, boolean hashJoinSecondOkay, boolean mergeJoinOkay) {
 		try {
 			// set statistics
 			OperatorResolver cr = getContractResolver(p);
-			GenericDataSourceBase<?,?> ordersSource = cr.getNode(ORDERS);
-			GenericDataSourceBase<?,?> lineItemSource = cr.getNode(LINEITEM);
-			SingleInputOperator<?,?,?> mapper = cr.getNode(MAPPER_NAME);
-			DualInputOperator<?,?,?,?> joiner = cr.getNode(JOIN_NAME);
+			GenericDataSourceBase<?, ?> ordersSource = cr.getNode(ORDERS);
+			GenericDataSourceBase<?, ?> lineItemSource = cr.getNode(LINEITEM);
+			SingleInputOperator<?, ?, ?> mapper = cr.getNode(MAPPER_NAME);
+			DualInputOperator<?, ?, ?, ?> joiner = cr.getNode(JOIN_NAME);
 			setSourceStatistics(ordersSource, orderSize, 100f);
 			setSourceStatistics(lineItemSource, lineitemSize, 140f);
 			mapper.getCompilerHints().setAvgOutputRecordSize(16f);
 			mapper.getCompilerHints().setFilterFactor(orderSelectivity);
 			joiner.getCompilerHints().setFilterFactor(joinSelectivity);
-			
+
 			// compile
 			final OptimizedPlan plan = compileWithStats(p);
 			final OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(plan);
-			
+
 			// get the nodes from the final plan
 			final SinkPlanNode sink = or.getNode(SINK);
 			final SingleInputPlanNode reducer = or.getNode(REDUCE_NAME);
@@ -192,13 +190,13 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 					(SingleInputPlanNode) reducer.getPredecessor() : null;
 			final DualInputPlanNode join = or.getNode(JOIN_NAME);
 			final SingleInputPlanNode filteringMapper = or.getNode(MAPPER_NAME);
-			
+
 			checkStandardStrategies(filteringMapper, join, combiner, reducer, sink);
-			
+
 			// check the possible variants and that the variant ia allowed in this specific setting
 			if (checkBroadcastShipStrategies(join, reducer, combiner)) {
 				Assert.assertTrue("Broadcast join incorrectly chosen.", broadcastOkay);
-				
+
 				if (checkHashJoinStrategies(join, reducer, true)) {
 					Assert.assertTrue("Hash join (build orders) incorrectly chosen", hashJoinFirstOkay);
 				} else if (checkHashJoinStrategies(join, reducer, false)) {
@@ -211,7 +209,7 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			}
 			else if (checkRepartitionShipStrategies(join, reducer, combiner)) {
 				Assert.assertTrue("Partitioned join incorrectly chosen.", partitionedOkay);
-				
+
 				if (checkHashJoinStrategies(join, reducer, true)) {
 					Assert.assertTrue("Hash join (build orders) incorrectly chosen", hashJoinFirstOkay);
 				} else if (checkHashJoinStrategies(join, reducer, false)) {
@@ -235,12 +233,11 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 	// ------------------------------------------------------------------------
 
 	private void checkStandardStrategies(SingleInputPlanNode map, DualInputPlanNode join, SingleInputPlanNode combiner,
-			SingleInputPlanNode reducer, SinkPlanNode sink)
-	{
+			SingleInputPlanNode reducer, SinkPlanNode sink) {
 		// check ship strategies that are always fix
 		Assert.assertEquals(ShipStrategyType.FORWARD, map.getInput().getShipStrategy());
 		Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
-		
+
 		// check the driver strategies that are always fix
 		Assert.assertEquals(DriverStrategy.FLAT_MAP, map.getDriverStrategy());
 		Assert.assertEquals(DriverStrategy.SORTED_GROUP_REDUCE, reducer.getDriverStrategy());
@@ -250,14 +247,13 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			Assert.assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy());
 		}
 	}
-	
+
 	private boolean checkBroadcastShipStrategies(DualInputPlanNode join, SingleInputPlanNode reducer,
-			SingleInputPlanNode combiner)
-	{
+			SingleInputPlanNode combiner) {
 		if (ShipStrategyType.BROADCAST == join.getInput1().getShipStrategy() &&
 			ShipStrategyType.FORWARD == join.getInput2().getShipStrategy() &&
-			ShipStrategyType.PARTITION_HASH == reducer.getInput().getShipStrategy())
-		{
+			ShipStrategyType.PARTITION_HASH == reducer.getInput().getShipStrategy()) {
+
 			// check combiner
 			Assert.assertNotNull("Plan should have a combiner", combiner);
 			Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
@@ -266,14 +262,13 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			return false;
 		}
 	}
-	
+
 	private boolean checkRepartitionShipStrategies(DualInputPlanNode join, SingleInputPlanNode reducer,
-			SingleInputPlanNode combiner)
-	{
+			SingleInputPlanNode combiner) {
 		if (ShipStrategyType.PARTITION_HASH == join.getInput1().getShipStrategy() &&
 			ShipStrategyType.PARTITION_HASH == join.getInput2().getShipStrategy() &&
-			ShipStrategyType.FORWARD == reducer.getInput().getShipStrategy())
-		{
+			ShipStrategyType.FORWARD == reducer.getInput().getShipStrategy()) {
+
 			// check combiner
 			Assert.assertNull("Plan should not have a combiner", combiner);
 			return true;
@@ -281,20 +276,20 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			return false;
 		}
 	}
-	
+
 	private boolean checkHashJoinStrategies(DualInputPlanNode join, SingleInputPlanNode reducer, boolean buildFirst) {
-		if ( (buildFirst && DriverStrategy.HYBRIDHASH_BUILD_FIRST == join.getDriverStrategy()) ||
-			 (!buildFirst && DriverStrategy.HYBRIDHASH_BUILD_SECOND == join.getDriverStrategy()) ) 
-		{
+		if ((buildFirst && DriverStrategy.HYBRIDHASH_BUILD_FIRST == join.getDriverStrategy()) ||
+			(!buildFirst && DriverStrategy.HYBRIDHASH_BUILD_SECOND == join.getDriverStrategy())) {
+
 			// driver keys
 			Assert.assertEquals(set0, join.getKeysForInput1());
 			Assert.assertEquals(set0, join.getKeysForInput2());
-			
+
 			// local strategies
 			Assert.assertEquals(LocalStrategy.NONE, join.getInput1().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.NONE, join.getInput2().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy());
-			
+
 			// local strategy keys
 			Assert.assertEquals(set01, reducer.getInput().getLocalStrategyKeys());
 			Assert.assertEquals(set01, reducer.getKeys(0));
@@ -304,18 +299,18 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			return false;
 		}
 	}
-	
+
 	private boolean checkBroadcastMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) {
 		if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) {
 			// driver keys
 			Assert.assertEquals(set0, join.getKeysForInput1());
 			Assert.assertEquals(set0, join.getKeysForInput2());
-			
+
 			// local strategies
 			Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.COMBININGSORT, reducer.getInput().getLocalStrategy());
-			
+
 			// local strategy keys
 			Assert.assertEquals(set0, join.getInput1().getLocalStrategyKeys());
 			Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys());
@@ -328,18 +323,18 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 			return false;
 		}
 	}
-	
+
 	private boolean checkRepartitionMergeJoin(DualInputPlanNode join, SingleInputPlanNode reducer) {
 		if (DriverStrategy.INNER_MERGE == join.getDriverStrategy()) {
 			// driver keys
 			Assert.assertEquals(set0, join.getKeysForInput1());
 			Assert.assertEquals(set0, join.getKeysForInput2());
-			
+
 			// local strategies
 			Assert.assertEquals(LocalStrategy.SORT, join.getInput1().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.SORT, join.getInput2().getLocalStrategy());
 			Assert.assertEquals(LocalStrategy.NONE, reducer.getInput().getLocalStrategy());
-			
+
 			// local strategy keys
 			Assert.assertEquals(set01, join.getInput1().getLocalStrategyKeys());
 			Assert.assertEquals(set0, join.getInput2().getLocalStrategyKeys());
@@ -357,30 +352,30 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 		PreviewPlanEnvironment env = new PreviewPlanEnvironment();
 		env.setAsContext();
 		try {
-			TCPH3(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE});
+			tcph3(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE});
 		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
-			Assert.fail("TCPH3 failed with an exception");
+			Assert.fail("tcph3 failed with an exception");
 		}
 		return env.getPlan();
 	}
 
-	public static void TCPH3(String[] args) throws Exception {
+	public static void tcph3(String[] args) throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(Integer.parseInt(args[0]));
 
 		//order id, order status, order data, order prio, ship prio
-		DataSet<Tuple5<Long, String, String, String, Integer>> orders
-				= env.readCsvFile(args[1])
+		DataSet<Tuple5<Long, String, String, String, Integer>> orders =
+				env.readCsvFile(args[1])
 				.fieldDelimiter("|").lineDelimiter("\n")
 				.includeFields("101011001").types(Long.class, String.class, String.class, String.class, Integer.class)
 				.name(ORDERS);
 
 		//order id, extended price
-		DataSet<Tuple2<Long, Double>> lineItems
-				= env.readCsvFile(args[2])
+		DataSet<Tuple2<Long, Double>> lineItems =
+				env.readCsvFile(args[2])
 				.fieldDelimiter("|").lineDelimiter("\n")
 				.includeFields("100001").types(Long.class, Double.class)
 				.name(LINEITEM);
@@ -397,7 +392,7 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 	}
 
 	@ForwardedFields("f0; f4->f1")
-	public static class FilterO implements FlatMapFunction<Tuple5<Long, String, String, String, Integer>, Tuple2<Long, Integer>> {
+	private static class FilterO implements FlatMapFunction<Tuple5<Long, String, String, String, Integer>, Tuple2<Long, Integer>> {
 		@Override
 		public void flatMap(Tuple5<Long, String, String, String, Integer> value, Collector<Tuple2<Long, Integer>> out) throws Exception {
 			// not going to be executed
@@ -405,7 +400,7 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 	}
 
 	@ForwardedFieldsFirst("f0; f1")
-	public static class JoinLiO implements FlatJoinFunction<Tuple2<Long, Integer>, Tuple2<Long, Double>, Tuple3<Long, Integer, Double>> {
+	private static class JoinLiO implements FlatJoinFunction<Tuple2<Long, Integer>, Tuple2<Long, Double>, Tuple3<Long, Integer, Double>> {
 		@Override
 		public void join(Tuple2<Long, Integer> first, Tuple2<Long, Double> second, Collector<Tuple3<Long, Integer, Double>> out) throws Exception {
 			// not going to be executed
@@ -413,10 +408,9 @@ public class RelationalQueryCompilerTest extends CompilerTestBase {
 	}
 
 	@ForwardedFields("f0; f1")
-	public static class AggLiO implements
-		GroupReduceFunction<Tuple3<Long, Integer, Double>, Tuple3<Long, Integer, Double>>,
-		GroupCombineFunction<Tuple3<Long, Integer, Double>, Tuple3<Long, Integer, Double>>
-	{
+	private static class AggLiO implements
+			GroupReduceFunction<Tuple3<Long, Integer, Double>, Tuple3<Long, Integer, Double>>,
+			GroupCombineFunction<Tuple3<Long, Integer, Double>, Tuple3<Long, Integer, Double>> {
 		@Override
 		public void reduce(Iterable<Tuple3<Long, Integer, Double>> values, Collector<Tuple3<Long, Integer, Double>> out) throws Exception {
 			// not going to be executed

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java
index ab49282..af780ab 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/WordCountCompilerTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.optimizer.examples;
 
-import java.util.Arrays;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.functions.MapFunction;
@@ -33,13 +31,19 @@ import org.apache.flink.optimizer.plan.Channel;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 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.optimizer.util.CompilerTestBase;
+
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.util.Arrays;
+
+/**
+ * Validate program compilation.
+ */
 public class WordCountCompilerTest extends CompilerTestBase {
 
 	private static final long serialVersionUID = 8988304231385358228L;
@@ -52,7 +56,7 @@ public class WordCountCompilerTest extends CompilerTestBase {
 		checkWordCount(true);
 		checkWordCount(false);
 	}
-	
+
 	private void checkWordCount(boolean estimates) {
 
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -81,8 +85,8 @@ public class WordCountCompilerTest extends CompilerTestBase {
 
 		OptimizedPlan plan;
 		if (estimates) {
-			GenericDataSourceBase<?,?> source = getContractResolver(p).getNode("Input Lines");
-			setSourceStatistics(source, 1024*1024*1024*1024L, 24f);
+			GenericDataSourceBase<?, ?> source = getContractResolver(p).getNode("Input Lines");
+			setSourceStatistics(source, 1024 * 1024 * 1024 * 1024L, 24f);
 			plan = compileWithStats(p);
 		} else {
 			plan = compileNoStats(p);
@@ -111,7 +115,7 @@ public class WordCountCompilerTest extends CompilerTestBase {
 		Assert.assertEquals(DriverStrategy.SORTED_GROUP_COMBINE, combiner.getDriverStrategy());
 		Assert.assertEquals(l, combiner.getKeys(0));
 		Assert.assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
-			
+
 	}
-	
+
 }


[07/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
new file mode 100644
index 0000000..dfb3efb
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java
@@ -0,0 +1,518 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link MapFunction} and {@link RichMapFunction}.
+ */
+@RunWith(Parameterized.class)
+public class MapITCase extends MultipleProgramsTestBase {
+
+	public MapITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testIdentityMapWithBasicType() throws Exception {
+		/*
+		 * Test identity map with basic type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> identityMapDs = ds.
+				map(new Mapper1());
+
+		List<String> result = identityMapDs.collect();
+
+		String expected = "Hi\n" +
+				"Hello\n" +
+				"Hello world\n" +
+				"Hello world, how are you?\n" +
+				"I am fine.\n" +
+				"Luke Skywalker\n" +
+				"Random comment\n" +
+				"LOL\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRuntimeContextAndExecutionConfigParams() throws Exception {
+		/*
+		 * Test identity map with basic type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().setNumberOfExecutionRetries(1000);
+		env.getConfig().setTaskCancellationInterval(50000);
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> identityMapDs = ds.
+			map(new RichMapFunction<String, String>() {
+				@Override
+				public String map(String value) throws Exception {
+					Assert.assertTrue(1000 == getRuntimeContext().getExecutionConfig().getNumberOfExecutionRetries());
+					Assert.assertTrue(50000 == getRuntimeContext().getExecutionConfig().getTaskCancellationInterval());
+					return value;
+				}
+			});
+
+		List<String> result = identityMapDs.collect();
+
+		String expected = "Hi\n" +
+			"Hello\n" +
+			"Hello world\n" +
+			"Hello world, how are you?\n" +
+			"I am fine.\n" +
+			"Luke Skywalker\n" +
+			"Random comment\n" +
+			"LOL\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper1 implements MapFunction<String, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String map(String value) throws Exception {
+			return value;
+		}
+	}
+
+	@Test
+	public void testIdentityMapWithTuple() throws Exception {
+		/*
+		 * Test identity map with a tuple
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds.
+				map(new Mapper2());
+
+		List<Tuple3<Integer, Long, String>> result = identityMapDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n" +
+				"6,3,Luke Skywalker\n" +
+				"7,4,Comment#1\n" +
+				"8,4,Comment#2\n" +
+				"9,4,Comment#3\n" +
+				"10,4,Comment#4\n" +
+				"11,5,Comment#5\n" +
+				"12,5,Comment#6\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" +
+				"15,5,Comment#9\n" +
+				"16,6,Comment#10\n" +
+				"17,6,Comment#11\n" +
+				"18,6,Comment#12\n" +
+				"19,6,Comment#13\n" +
+				"20,6,Comment#14\n" +
+				"21,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Mapper2 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
+				throws Exception {
+			return value;
+		}
+	}
+
+	@Test
+	public void testTypeConversionMapperCustomToTuple() throws Exception {
+		/*
+		 * Test type conversion mapper (Custom -> Tuple)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> typeConversionMapDs = ds.
+				map(new Mapper3());
+
+		List<Tuple3<Integer, Long, String>> result = typeConversionMapDs.collect();
+
+		String expected = "1,0,Hi\n" +
+				"2,1,Hello\n" +
+				"2,2,Hello world\n" +
+				"3,3,Hello world, how are you?\n" +
+				"3,4,I am fine.\n" +
+				"3,5,Luke Skywalker\n" +
+				"4,6,Comment#1\n" +
+				"4,7,Comment#2\n" +
+				"4,8,Comment#3\n" +
+				"4,9,Comment#4\n" +
+				"5,10,Comment#5\n" +
+				"5,11,Comment#6\n" +
+				"5,12,Comment#7\n" +
+				"5,13,Comment#8\n" +
+				"5,14,Comment#9\n" +
+				"6,15,Comment#10\n" +
+				"6,16,Comment#11\n" +
+				"6,17,Comment#12\n" +
+				"6,18,Comment#13\n" +
+				"6,19,Comment#14\n" +
+				"6,20,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Mapper3 implements MapFunction<CustomType, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
+
+		@Override
+		public Tuple3<Integer, Long, String> map(CustomType value) throws Exception {
+			out.setField(value.myInt, 0);
+			out.setField(value.myLong, 1);
+			out.setField(value.myString, 2);
+			return out;
+		}
+	}
+
+	@Test
+	public void testTypeConversionMapperTupleToBasic() throws Exception {
+		/*
+		 * Test type conversion mapper (Tuple -> Basic)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<String> typeConversionMapDs = ds.
+				map(new Mapper4());
+
+		List<String> result = typeConversionMapDs.collect();
+
+		String expected = "Hi\n" + "Hello\n" + "Hello world\n" +
+				"Hello world, how are you?\n" +
+				"I am fine.\n" + "Luke Skywalker\n" +
+				"Comment#1\n" +	"Comment#2\n" +
+				"Comment#3\n" +	"Comment#4\n" +
+				"Comment#5\n" +	"Comment#6\n" +
+				"Comment#7\n" + "Comment#8\n" +
+				"Comment#9\n" +	"Comment#10\n" +
+				"Comment#11\n" + "Comment#12\n" +
+				"Comment#13\n" + "Comment#14\n" +
+				"Comment#15\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper4 implements MapFunction<Tuple3<Integer, Long, String>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String map(Tuple3<Integer, Long, String> value) throws Exception {
+			return value.getField(2);
+		}
+	}
+
+	@Test
+	public void testMapperOnTupleIncrementIntegerFieldReorderSecondAndThirdFields() throws
+	Exception {
+		/*
+		 * Test mapper on tuple - Increment Integer field, reorder second and third fields
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, String, Long>> tupleMapDs = ds.
+				map(new Mapper5());
+
+		List<Tuple3<Integer, String, Long>> result = tupleMapDs.collect();
+
+		String expected = "2,Hi,1\n" +
+				"3,Hello,2\n" +
+				"4,Hello world,2\n" +
+				"5,Hello world, how are you?,3\n" +
+				"6,I am fine.,3\n" +
+				"7,Luke Skywalker,3\n" +
+				"8,Comment#1,4\n" +
+				"9,Comment#2,4\n" +
+				"10,Comment#3,4\n" +
+				"11,Comment#4,4\n" +
+				"12,Comment#5,5\n" +
+				"13,Comment#6,5\n" +
+				"14,Comment#7,5\n" +
+				"15,Comment#8,5\n" +
+				"16,Comment#9,5\n" +
+				"17,Comment#10,6\n" +
+				"18,Comment#11,6\n" +
+				"19,Comment#12,6\n" +
+				"20,Comment#13,6\n" +
+				"21,Comment#14,6\n" +
+				"22,Comment#15,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Mapper5 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, String, Long>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, String, Long> out = new Tuple3<Integer, String, Long>();
+
+		@Override
+		public Tuple3<Integer, String, Long> map(Tuple3<Integer, Long, String> value)
+				throws Exception {
+			Integer incr = Integer.valueOf(value.f0.intValue() + 1);
+			out.setFields(incr, value.f2, value.f1);
+			return out;
+		}
+	}
+
+	@Test
+	public void testMapperOnCustomLowercaseString() throws Exception {
+		/*
+		 * Test mapper on Custom - lowercase myString
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> customMapDs = ds.
+				map(new Mapper6());
+
+		List<CustomType> result = customMapDs.collect();
+
+		String expected = "1,0,hi\n" +
+				"2,1,hello\n" +
+				"2,2,hello world\n" +
+				"3,3,hello world, how are you?\n" +
+				"3,4,i am fine.\n" +
+				"3,5,luke skywalker\n" +
+				"4,6,comment#1\n" +
+				"4,7,comment#2\n" +
+				"4,8,comment#3\n" +
+				"4,9,comment#4\n" +
+				"5,10,comment#5\n" +
+				"5,11,comment#6\n" +
+				"5,12,comment#7\n" +
+				"5,13,comment#8\n" +
+				"5,14,comment#9\n" +
+				"6,15,comment#10\n" +
+				"6,16,comment#11\n" +
+				"6,17,comment#12\n" +
+				"6,18,comment#13\n" +
+				"6,19,comment#14\n" +
+				"6,20,comment#15\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper6 implements MapFunction<CustomType, CustomType> {
+		private static final long serialVersionUID = 1L;
+		private final CustomType out = new CustomType();
+
+		@Override
+		public CustomType map(CustomType value) throws Exception {
+			out.myInt = value.myInt;
+			out.myLong = value.myLong;
+			out.myString = value.myString.toLowerCase();
+			return out;
+		}
+	}
+
+	@Test
+	public void test() throws Exception {
+		/*
+		 * Test mapper if UDF returns input object - increment first field of a tuple
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> inputObjMapDs = ds.
+				map(new Mapper7());
+
+		List<Tuple3<Integer, Long, String>> result = inputObjMapDs.collect();
+
+		String expected = "2,1,Hi\n" +
+				"3,2,Hello\n" +
+				"4,2,Hello world\n" +
+				"5,3,Hello world, how are you?\n" +
+				"6,3,I am fine.\n" +
+				"7,3,Luke Skywalker\n" +
+				"8,4,Comment#1\n" +
+				"9,4,Comment#2\n" +
+				"10,4,Comment#3\n" +
+				"11,4,Comment#4\n" +
+				"12,5,Comment#5\n" +
+				"13,5,Comment#6\n" +
+				"14,5,Comment#7\n" +
+				"15,5,Comment#8\n" +
+				"16,5,Comment#9\n" +
+				"17,6,Comment#10\n" +
+				"18,6,Comment#11\n" +
+				"19,6,Comment#12\n" +
+				"20,6,Comment#13\n" +
+				"21,6,Comment#14\n" +
+				"22,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Mapper7 implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
+				throws Exception {
+			Integer incr = Integer.valueOf(value.f0.intValue() + 1);
+			value.setField(incr, 0);
+			return value;
+		}
+	}
+
+	@Test
+	public void testMapWithBroadcastSet() throws Exception {
+		/*
+		 * Test map with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> bcMapDs = ds.
+				map(new RichMapper1()).withBroadcastSet(ints, "ints");
+		List<Tuple3<Integer, Long, String>> result = bcMapDs.collect();
+
+		String expected = "55,1,Hi\n" +
+				"55,2,Hello\n" +
+				"55,2,Hello world\n" +
+				"55,3,Hello world, how are you?\n" +
+				"55,3,I am fine.\n" +
+				"55,3,Luke Skywalker\n" +
+				"55,4,Comment#1\n" +
+				"55,4,Comment#2\n" +
+				"55,4,Comment#3\n" +
+				"55,4,Comment#4\n" +
+				"55,5,Comment#5\n" +
+				"55,5,Comment#6\n" +
+				"55,5,Comment#7\n" +
+				"55,5,Comment#8\n" +
+				"55,5,Comment#9\n" +
+				"55,6,Comment#10\n" +
+				"55,6,Comment#11\n" +
+				"55,6,Comment#12\n" +
+				"55,6,Comment#13\n" +
+				"55,6,Comment#14\n" +
+				"55,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichMapper1 extends RichMapFunction<Tuple3<Integer, Long, String>,
+	Tuple3<Integer,	Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
+		private Integer f2Replace = 0;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			f2Replace = sum;
+		}
+
+		@Override
+		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value)
+				throws Exception {
+			out.setFields(f2Replace, value.f1, value.f2);
+			return out;
+		}
+	}
+
+	static final String TEST_KEY = "testVariable";
+	static final int TEST_VALUE = 666;
+
+	@Test
+	public void testPassingConfigurationObject() throws Exception {
+		/*
+		 * Test passing configuration object.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		Configuration conf = new Configuration();
+		conf.setInteger(TEST_KEY, TEST_VALUE);
+		DataSet<Tuple3<Integer, Long, String>> bcMapDs = ds.
+				map(new RichMapper2()).withParameters(conf);
+		List<Tuple3<Integer, Long, String>> result = bcMapDs.collect();
+
+		String expected = "1,1,Hi\n"
+				+ "2,2,Hello\n"
+				+ "3,2,Hello world";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichMapper2 extends RichMapFunction<Tuple3<Integer, Long, String>,
+	Tuple3<Integer,	Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void open(Configuration config) {
+			int val = config.getInteger(TEST_KEY, -1);
+			Assert.assertEquals(TEST_VALUE, val);
+		}
+
+		@Override
+		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value) {
+			return value;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java
new file mode 100644
index 0000000..bb7f705
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapPartitionITCase.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Integration tests for {@link MapPartitionFunction}.
+ */
+@SuppressWarnings("serial")
+public class MapPartitionITCase extends JavaProgramTestBase {
+
+	private static final String IN = "1 1\n2 2\n2 8\n4 4\n4 4\n6 6\n7 7\n8 8\n"
+			+ "1 1\n2 2\n2 2\n4 4\n4 4\n6 3\n5 9\n8 8\n1 1\n2 2\n2 2\n3 0\n4 4\n"
+			+ "5 9\n7 7\n8 8\n1 1\n9 1\n5 9\n4 4\n4 4\n6 6\n7 7\n8 8\n";
+
+	private static final String RESULT = "1 11\n2 12\n4 14\n4 14\n1 11\n2 12\n2 12\n4 14\n4 14\n3 16\n1 11\n2 12\n2 12\n0 13\n4 14\n1 11\n4 14\n4 14\n";
+
+	private List<Tuple2<String, String>> input = new ArrayList<>();
+
+	private List<Tuple2<String, Integer>> expected = new ArrayList<>();
+
+	private List<Tuple2<String, Integer>> result = new ArrayList<>();
+
+	@Override
+	protected void preSubmit() throws Exception {
+
+		// create input
+		for (String s :IN.split("\n")) {
+			String[] fields = s.split(" ");
+			input.add(new Tuple2<String, String>(fields[0], fields[1]));
+		}
+
+		// create expected
+		for (String s : RESULT.split("\n")) {
+			String[] fields = s.split(" ");
+			expected.add(new Tuple2<String, Integer>(fields[0], Integer.parseInt(fields[1])));
+		}
+
+	}
+
+	@Override
+	protected void postSubmit() {
+		compareResultCollections(expected, result, new TestBaseUtils.TupleComparator<Tuple2<String, Integer>>());
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<String, String>> data = env.fromCollection(input);
+
+		data.mapPartition(new TestMapPartition()).output(new LocalCollectionOutputFormat<Tuple2<String, Integer>>(result));
+
+		env.execute();
+	}
+
+	private static class TestMapPartition implements MapPartitionFunction<Tuple2<String, String>, Tuple2<String, Integer>> {
+
+		@Override
+		public void mapPartition(Iterable<Tuple2<String, String>> values, Collector<Tuple2<String, Integer>> out) {
+			for (Tuple2<String, String> value : values) {
+				String keyString = value.f0;
+				String valueString = value.f1;
+
+				int keyInt = Integer.parseInt(keyString);
+				int valueInt = Integer.parseInt(valueString);
+
+				if (keyInt + valueInt < 10) {
+					out.collect(new Tuple2<String, Integer>(valueString, keyInt + 10));
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java
new file mode 100644
index 0000000..37659d2
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ObjectReuseITCase.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * These check whether the object-reuse execution mode does really reuse objects.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class ObjectReuseITCase extends MultipleProgramsTestBase {
+
+	private static final List<Tuple2<String, Integer>> REDUCE_DATA =
+		Arrays.asList(
+			new Tuple2<>("a", 1), new Tuple2<>("a", 2),
+			new Tuple2<>("a", 3), new Tuple2<>("a", 4),
+			new Tuple2<>("a", 50));
+
+	private static final List<Tuple2<String, Integer>> GROUP_REDUCE_DATA =
+		Arrays.asList(
+			new Tuple2<>("a", 1), new Tuple2<>("a", 2),
+			new Tuple2<>("a", 3), new Tuple2<>("a", 4),
+			new Tuple2<>("a", 5));
+
+	private final boolean objectReuse;
+
+	public ObjectReuseITCase(boolean objectReuse) {
+		super(TestExecutionMode.CLUSTER);
+		this.objectReuse = objectReuse;
+	}
+
+	@Test
+	public void testKeyedReduce() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		if (objectReuse) {
+			env.getConfig().enableObjectReuse();
+		} else {
+			env.getConfig().disableObjectReuse();
+		}
+
+		DataSet<Tuple2<String, Integer>> input = env.fromCollection(REDUCE_DATA);
+
+		DataSet<Tuple2<String, Integer>> result = input
+			.groupBy(0)
+			.reduce(new ReduceFunction<Tuple2<String, Integer>>() {
+
+				@Override
+				public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) {
+					value2.f1 += value1.f1;
+					return value2;
+				}
+			});
+
+		Tuple2<String, Integer> res = result.collect().get(0);
+		assertEquals(new Tuple2<>("a", 60), res);
+	}
+
+	@Test
+	public void testGlobalReduce() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		if (objectReuse) {
+			env.getConfig().enableObjectReuse();
+		} else {
+			env.getConfig().disableObjectReuse();
+		}
+
+		DataSet<Tuple2<String, Integer>> input = env.fromCollection(REDUCE_DATA);
+
+		DataSet<Tuple2<String, Integer>> result = input.reduce(
+			new ReduceFunction<Tuple2<String, Integer>>() {
+
+				@Override
+				public Tuple2<String, Integer> reduce(
+						Tuple2<String, Integer> value1,
+						Tuple2<String, Integer> value2) {
+
+					if (value1.f1 % 3 == 0) {
+						value1.f1 += value2.f1;
+						return value1;
+					} else {
+						value2.f1 += value1.f1;
+						return value2;
+					}
+				}
+
+			});
+
+		Tuple2<String, Integer> res = result.collect().get(0);
+		assertEquals(new Tuple2<>("a", 60), res);
+	}
+
+	@Test
+	public void testKeyedGroupReduce() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		if (objectReuse) {
+			env.getConfig().enableObjectReuse();
+		} else {
+			env.getConfig().disableObjectReuse();
+		}
+
+		DataSet<Tuple2<String, Integer>> input = env.fromCollection(GROUP_REDUCE_DATA);
+
+		DataSet<Tuple2<String, Integer>> result = input.groupBy(0).reduceGroup(
+			new GroupReduceFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+
+				@Override
+				public void reduce(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
+					List<Tuple2<String, Integer>> list = new ArrayList<>();
+					for (Tuple2<String, Integer> val : values) {
+						list.add(val);
+					}
+
+					for (Tuple2<String, Integer> val : list) {
+						out.collect(val);
+					}
+				}
+			});
+
+		List<Tuple2<String, Integer>> is = result.collect();
+		Collections.sort(is, new TupleComparator<Tuple2<String, Integer>>());
+
+		List<Tuple2<String, Integer>> expected = env.getConfig().isObjectReuseEnabled() ?
+			Arrays.asList(new Tuple2<>("a", 4), new Tuple2<>("a", 4),
+				new Tuple2<>("a", 5), new Tuple2<>("a", 5), new Tuple2<>("a", 5)) :
+			Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2),
+				new Tuple2<>("a", 3), new Tuple2<>("a", 4), new Tuple2<>("a", 5));
+
+		assertEquals(expected, is);
+	}
+
+	@Test
+	public void testGlobalGroupReduce() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		if (objectReuse) {
+			env.getConfig().enableObjectReuse();
+		} else {
+			env.getConfig().disableObjectReuse();
+		}
+
+		DataSet<Tuple2<String, Integer>> input = env.fromCollection(GROUP_REDUCE_DATA);
+
+		DataSet<Tuple2<String, Integer>> result = input.reduceGroup(
+			new GroupReduceFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+
+				@Override
+				public void reduce(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
+					List<Tuple2<String, Integer>> list = new ArrayList<>();
+					for (Tuple2<String, Integer> val : values) {
+						list.add(val);
+					}
+
+					for (Tuple2<String, Integer> val : list) {
+						out.collect(val);
+					}
+				}
+			});
+
+		List<Tuple2<String, Integer>> is = result.collect();
+		Collections.sort(is, new TupleComparator<Tuple2<String, Integer>>());
+
+		List<Tuple2<String, Integer>> expected = env.getConfig().isObjectReuseEnabled() ?
+			Arrays.asList(new Tuple2<>("a", 4), new Tuple2<>("a", 4),
+				new Tuple2<>("a", 5), new Tuple2<>("a", 5), new Tuple2<>("a", 5)) :
+			Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2),
+				new Tuple2<>("a", 3), new Tuple2<>("a", 4), new Tuple2<>("a", 5));
+
+		assertEquals(expected, is);
+	}
+
+	@Parameterized.Parameters(name = "Execution mode = CLUSTER, Reuse = {0}")
+	public static Collection<Object[]> executionModes() {
+		return Arrays.asList(
+			new Object[] { false, },
+			new Object[] { true });
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java
new file mode 100644
index 0000000..31e5062
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java
@@ -0,0 +1,682 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link JoinFunction}, {@link FlatJoinFunction},
+ * and {@link RichFlatJoinFunction}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class OuterJoinITCase extends MultipleProgramsTestBase {
+
+	public OuterJoinITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testLeftOuterJoin1() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testLeftOuterJoin2() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test
+	public void testLeftOuterJoin3() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test
+	public void testLeftOuterJoin4() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testLeftOuterJoin5() throws Exception {
+		testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+	private void testLeftOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.leftOuterJoin(ds2, hint)
+						.where(0)
+						.equalTo(0)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello,Hallo Welt wie\n" +
+				"Hello world,null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testRightOuterJoin1() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testRightOuterJoin2() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test
+	public void testRightOuterJoin3() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test
+	public void testRightOuterJoin4() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+	@Test (expected = InvalidProgramException.class)
+	public void testRightOuterJoin5() throws Exception {
+		testRightOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	private void testRightOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.rightOuterJoin(ds2, hint)
+						.where(1)
+						.equalTo(1)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"null,Hallo Welt wie\n" +
+				"Hello world,Hallo Welt\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testFullOuterJoin1() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_SORT_MERGE);
+	}
+
+	@Test
+	public void testFullOuterJoin2() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_FIRST);
+	}
+
+	@Test
+	public void testFullOuterJoin3() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.REPARTITION_HASH_SECOND);
+	}
+
+	@Test (expected = InvalidProgramException.class)
+	public void testFullOuterJoin4() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_FIRST);
+	}
+
+	@Test (expected = InvalidProgramException.class)
+	public void testFullOuterJoin5() throws Exception {
+		testFullOuterJoinOnTuplesWithKeyPositions(JoinHint.BROADCAST_HASH_SECOND);
+	}
+
+	private void testFullOuterJoinOnTuplesWithKeyPositions(JoinHint hint) throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2, hint)
+						.where(0)
+						.equalTo(2)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "null,Hallo\n" +
+				"Hi,Hallo Welt\n" +
+				"Hello,Hallo Welt wie\n" +
+				"Hello world,null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinOnTuplesWithCompositeKeyPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with multiple key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(0, 1)
+						.equalTo(0, 4)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,null\n" +
+				"null,Hallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithBroadcastSet() throws Exception {
+		/*
+		 * Join with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple3<String, String, Integer>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(1)
+						.equalTo(4)
+						.with(new T3T5BCJoin())
+						.withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<String, String, Integer>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo,55\n" +
+				"Hi,Hallo Welt wie,55\n" +
+				"Hello,Hallo Welt,55\n" +
+				"Hello world,Hallo Welt,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithMixedKeyTypes1() throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(new KeySelector1())
+						.equalTo(0)
+						.with(new CustT3Join());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hi\n" +
+				"Hello,Hello\n" +
+				"Hello world,Hello\n" +
+				"null,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	private static class KeySelector1 implements KeySelector<CustomType, Integer> {
+		@Override
+		public Integer getKey(CustomType value) {
+			return value.myInt;
+		}
+	}
+
+	@Test
+	public void testJoinWithMixedKeyTypes2()
+			throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(1)
+						.equalTo(new KeySelector2())
+						.with(new T3CustJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "null,Hi\n" +
+				"Hi,Hello\n" +
+				"Hello,Hello world\n" +
+				"Hello world,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<CustomType, Long> {
+		@Override
+		public Long getKey(CustomType value) {
+			return value.myLong;
+		}
+	}
+
+	@Test
+	public void testJoinWithTupleReturningKeySelectors() throws Exception {
+		/*
+		 * UDF Join on tuples with tuple-returning key selectors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(new KeySelector3()) //0, 1
+						.equalTo(new KeySelector4()) // 0, 4
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,null\n" +
+				"null,Hallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector3 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+			return new Tuple2<>(t.f0, t.f1);
+		}
+	}
+
+	private static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testJoinWithNestedKeyExpression1() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected using a string)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber")
+						.equalTo("f6")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithNestedKeyExpression2() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected as an integer)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber")
+						.equalTo(6) // <--- difference!
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithCompositeKeyExpressions() throws Exception {
+		/*
+		 * selecting multiple fields using expression language
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber", "number", "str")
+						.equalTo("f6", "f0", "f1")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTuple() throws Exception {
+		/*
+		 * nested into tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedPojo.longNumber", "number", "nestedTupleWithCustom.f0")
+						.equalTo("f6", "f0", "f2")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTupleIntoPojo() throws Exception {
+		/*
+		 * nested into tuple into pojo
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("nestedTupleWithCustom.f0", "nestedTupleWithCustom.f1.myInt", "nestedTupleWithCustom.f1.myLong")
+						.equalTo("f2", "f3", "f4")
+						.with(new ProjectBothFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNonPojoToVerifyFullTupleKeys() throws Exception {
+		/*
+		 * Non-POJO test to verify that full-tuple keys are working.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where(0)
+						.equalTo("f0.f0", "f0.f1") // key is now Tuple2<Integer, Integer>
+						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	@Test
+	public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception {
+		/*
+		 * Non-POJO test to verify "nested" tuple-element selection.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("f0.f0")
+						.equalTo("f0.f0") // key is now Integer from Tuple2<Integer, Integer>
+						.with(new ProjectBothFunction<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testFullPojoWithFullTuple() throws Exception {
+		/*
+		 * full pojo with full tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env);
+		DataSet<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> joinDs =
+				ds1.fullOuterJoin(ds2)
+						.where("*")
+						.equalTo("*")
+						.with(new ProjectBothFunction<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>());
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
+				"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +
+				"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Integer> ds2 = env.fromElements(1, 2);
+
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Integer>> joinDs = ds1
+				.fullOuterJoin(ds2)
+				.where(0)
+				.equalTo("*")
+				.with(new ProjectBothFunction<Tuple3<Integer, Long, String>, Integer>())
+				.returns("Tuple2<java.lang.Object,java.lang.Object>");
+
+		List<Tuple2<Tuple3<Integer, Long, String>, Integer>> result = joinDs.collect();
+
+		String expected = "(1,1,Hi),1\n" +
+				"(2,2,Hello),2\n" +
+				"(3,2,Hello world),null\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType2() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ds1 = env.fromElements(1, 2);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+
+		DataSet<Tuple2<Integer, Tuple3<Integer, Long, String>>> joinDs = ds1
+				.fullOuterJoin(ds2)
+				.where("*")
+				.equalTo(0)
+				.with(new ProjectBothFunction<Integer, Tuple3<Integer, Long, String>>())
+				.returns("Tuple2<java.lang.Object,java.lang.Object>");
+
+		List<Tuple2<Integer, Tuple3<Integer, Long, String>>> result = joinDs.collect();
+
+		String expected = "1,(1,1,Hi)\n" +
+				"2,(2,2,Hello)\n" +
+				"null,(3,2,Hello world)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class T3T5FlatJoin implements FlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<String, String>> {
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second,
+				Collector<Tuple2<String, String>> out) {
+
+			out.collect(new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.f3));
+		}
+
+	}
+
+	private static class T3T5BCJoin extends RichFlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<String, String, Integer>> {
+
+		private int broadcast;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			broadcast = sum;
+		}
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first, Tuple5<Integer, Long, Integer, String, Long> second,
+				Collector<Tuple3<String, String, Integer>> out) throws Exception {
+			out.collect(new Tuple3<>(first == null ? null : first.f2, second == null ? null : second.f3, broadcast));
+		}
+	}
+
+	private static class T3CustJoin implements JoinFunction<Tuple3<Integer, Long, String>, CustomType, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(Tuple3<Integer, Long, String> first,
+				CustomType second) {
+
+			return new Tuple2<>(first == null ? null : first.f2, second == null ? null : second.myString);
+		}
+	}
+
+	private static class CustT3Join implements JoinFunction<CustomType, Tuple3<Integer, Long, String>, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(CustomType first, Tuple3<Integer, Long, String> second) {
+
+			return new Tuple2<>(first == null ? null : first.myString, second == null ? null : second.f2);
+		}
+	}
+
+	/**
+	 * Deliberately untyped join function, which emits a Tuple2 of the left and right side.
+	 */
+	private static class ProjectBothFunction<IN1, IN2> implements JoinFunction<IN1, IN2, Tuple2<IN1, IN2>> {
+		@Override
+		public Tuple2<IN1, IN2> join(IN1 first, IN2 second) throws Exception {
+			return new Tuple2<>(first, second);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java
new file mode 100644
index 0000000..5e88430
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/PartitionITCase.java
@@ -0,0 +1,847 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.AggregateOperator;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Integration tests for {@link MapPartitionFunction}.
+ */
+@RunWith(Parameterized.class)
+@SuppressWarnings("serial")
+public class PartitionITCase extends MultipleProgramsTestBase {
+
+	public PartitionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testHashPartitionByKeyField() throws Exception {
+		/*
+		 * Test hash partition by key field
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+				.partitionByHash(1)
+				.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+				"2\n" +
+				"3\n" +
+				"4\n" +
+				"5\n" +
+				"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRangePartitionByKeyField() throws Exception {
+		/*
+		 * Test range partition by key field
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+			.partitionByRange(1)
+			.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+			"2\n" +
+			"3\n" +
+			"4\n" +
+			"5\n" +
+			"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testHashPartitionByKeyField2() throws Exception {
+		/*
+		 * Test hash partition by key field
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		AggregateOperator<Tuple3<Integer, Long, String>> sum = ds
+			.map(new PrefixMapper())
+			.partitionByHash(1, 2)
+			.groupBy(1, 2)
+			.sum(0);
+
+		List<Tuple3<Integer, Long, String>> result = sum.collect();
+
+		String expected = "(1,1,Hi)\n" +
+			"(5,2,Hello)\n" +
+			"(4,3,Hello)\n" +
+			"(5,3,I am )\n" +
+			"(6,3,Luke )\n" +
+			"(34,4,Comme)\n" +
+			"(65,5,Comme)\n" +
+			"(111,6,Comme)";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRangePartitionByKeyField2() throws Exception {
+		/*
+		 * Test range partition by key field
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		AggregateOperator<Tuple3<Integer, Long, String>> sum = ds
+			.map(new PrefixMapper())
+			.partitionByRange(1, 2)
+			.groupBy(1, 2)
+			.sum(0);
+
+		List<Tuple3<Integer, Long, String>> result = sum.collect();
+
+		String expected = "(1,1,Hi)\n" +
+		"(5,2,Hello)\n" +
+		"(4,3,Hello)\n" +
+		"(5,3,I am )\n" +
+		"(6,3,Luke )\n" +
+		"(34,4,Comme)\n" +
+		"(65,5,Comme)\n" +
+		"(111,6,Comme)";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testHashPartitionOfAtomicType() throws Exception {
+		/*
+		 * Test hash partition of atomic type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Long> uniqLongs = env.generateSequence(1, 6)
+			.union(env.generateSequence(1, 6))
+			.rebalance()
+			.partitionByHash("*")
+			.mapPartition(new UniqueLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+			"2\n" +
+			"3\n" +
+			"4\n" +
+			"5\n" +
+			"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRangePartitionOfAtomicType() throws Exception {
+		/*
+		 * Test range partition of atomic type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Long> uniqLongs = env.generateSequence(1, 6)
+			.union(env.generateSequence(1, 6))
+			.rebalance()
+			.partitionByRange("*")
+			.mapPartition(new UniqueLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+			"2\n" +
+			"3\n" +
+			"4\n" +
+			"5\n" +
+			"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testHashPartitionByKeySelector() throws Exception {
+		/*
+		 * Test hash partition by key selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+				.partitionByHash(new KeySelector1())
+				.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+				"2\n" +
+				"3\n" +
+				"4\n" +
+				"5\n" +
+				"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class PrefixMapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		@Override
+		public Tuple3<Integer, Long, String> map(Tuple3<Integer, Long, String> value) throws Exception {
+			if (value.f2.length() > 5) {
+				value.f2 = value.f2.substring(0, 5);
+			}
+			return value;
+		}
+	}
+
+	@Test
+	public void testRangePartitionByKeySelector() throws Exception {
+		/*
+		 * Test range partition by key selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+			.partitionByRange(new KeySelector1())
+			.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+			"2\n" +
+			"3\n" +
+			"4\n" +
+			"5\n" +
+			"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector1 implements KeySelector<Tuple3<Integer, Long, String>, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long getKey(Tuple3<Integer, Long, String> value) throws Exception {
+			return value.f1;
+		}
+
+	}
+
+	@Test
+	public void testForcedRebalancing() throws Exception {
+		/*
+		 * Test forced rebalancing
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// generate some number in parallel
+		DataSet<Long> ds = env.generateSequence(1, 3000);
+		DataSet<Tuple2<Integer, Integer>> uniqLongs = ds
+				// introduce some partition skew by filtering
+				.filter(new Filter1())
+				// rebalance
+				.rebalance()
+				// count values in each partition
+				.map(new PartitionIndexMapper())
+				.groupBy(0)
+				.reduce(new Reducer1())
+				// round counts to mitigate runtime scheduling effects (lazy split assignment)
+				.map(new Mapper1());
+
+		List<Tuple2<Integer, Integer>> result = uniqLongs.collect();
+
+		StringBuilder expected = new StringBuilder();
+		int numPerPartition = 2220 / env.getParallelism() / 10;
+		for (int i = 0; i < env.getParallelism(); i++) {
+			expected.append('(').append(i).append(',')
+			.append(numPerPartition).append(")\n");
+		}
+
+		compareResultAsText(result, expected.toString());
+	}
+
+	private static class Filter1 implements FilterFunction<Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Long value) throws Exception {
+			return value > 780;
+		}
+	}
+
+	private static class Reducer1 implements ReduceFunction<Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> v1, Tuple2<Integer, Integer> v2) {
+			return new Tuple2<>(v1.f0, v1.f1 + v2.f1);
+		}
+	}
+
+	private static class Mapper1 implements MapFunction<Tuple2<Integer, Integer>, Tuple2<Integer,
+	Integer>>{
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) throws Exception {
+			value.f1 = (value.f1 / 10);
+			return value;
+		}
+
+	}
+
+	@Test
+	public void testHashPartitionByKeyFieldAndDifferentParallelism() throws Exception {
+		/*
+		 * Test hash partition by key field and different parallelism
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+				.partitionByHash(1).setParallelism(4)
+				.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+				"2\n" +
+				"3\n" +
+				"4\n" +
+				"5\n" +
+				"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRangePartitionByKeyFieldAndDifferentParallelism() throws Exception {
+		/*
+		 * Test range partition by key field and different parallelism
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Long> uniqLongs = ds
+			.partitionByRange(1).setParallelism(4)
+			.mapPartition(new UniqueTupleLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "1\n" +
+			"2\n" +
+			"3\n" +
+			"4\n" +
+			"5\n" +
+			"6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testHashPartitionWithKeyExpression() throws Exception {
+		/*
+		 * Test hash partition with key expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
+		DataSet<Long> uniqLongs = ds
+				.partitionByHash("nestedPojo.longNumber").setParallelism(4)
+				.mapPartition(new UniqueNestedPojoLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "10000\n" +
+				"20000\n" +
+				"30000\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testRangePartitionWithKeyExpression() throws Exception {
+		/*
+		 * Test range partition with key expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
+		DataSet<Long> uniqLongs = ds
+			.partitionByRange("nestedPojo.longNumber").setParallelism(4)
+			.mapPartition(new UniqueNestedPojoLongMapper());
+		List<Long> result = uniqLongs.collect();
+
+		String expected = "10000\n" +
+			"20000\n" +
+			"30000\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class UniqueTupleLongMapper implements MapPartitionFunction<Tuple3<Integer, Long, String>, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void mapPartition(Iterable<Tuple3<Integer, Long, String>> records, Collector<Long> out) throws Exception {
+			HashSet<Long> uniq = new HashSet<>();
+			for (Tuple3<Integer, Long, String> t : records) {
+				uniq.add(t.f1);
+			}
+			for (Long l : uniq) {
+				out.collect(l);
+			}
+		}
+	}
+
+	private static class UniqueLongMapper implements MapPartitionFunction<Long, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void mapPartition(Iterable<Long> longs, Collector<Long> out) throws Exception {
+			HashSet<Long> uniq = new HashSet<>();
+			for (Long l : longs) {
+				uniq.add(l);
+			}
+			for (Long l : uniq) {
+				out.collect(l);
+			}
+		}
+	}
+
+	private static class UniqueNestedPojoLongMapper implements MapPartitionFunction<POJO, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void mapPartition(Iterable<POJO> records, Collector<Long> out) throws Exception {
+			HashSet<Long> uniq = new HashSet<>();
+			for (POJO t : records) {
+				uniq.add(t.nestedPojo.longNumber);
+			}
+			for (Long l : uniq) {
+				out.collect(l);
+			}
+		}
+	}
+
+	private static class PartitionIndexMapper extends RichMapFunction<Long, Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> map(Long value) throws Exception {
+			return new Tuple2<>(this.getRuntimeContext().getIndexOfThisSubtask(), 1);
+		}
+	}
+
+	@Test
+	public void testRangePartitionerOnSequenceData() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSource<Long> dataSource = env.generateSequence(0, 10000);
+		KeySelector<Long, Long> keyExtractor = new ObjectSelfKeySelector();
+
+		MapPartitionFunction<Long, Tuple2<Long, Long>> minMaxSelector = new MinMaxSelector<>(new LongComparator(true));
+
+		Comparator<Tuple2<Long, Long>> tuple2Comparator = new Tuple2Comparator(new LongComparator(true));
+
+		List<Tuple2<Long, Long>> collected = dataSource.partitionByRange(keyExtractor).mapPartition(minMaxSelector).collect();
+		Collections.sort(collected, tuple2Comparator);
+
+		long previousMax = -1;
+		for (Tuple2<Long, Long> tuple2 : collected) {
+			if (previousMax == -1) {
+				previousMax = tuple2.f1;
+			} else {
+				long currentMin = tuple2.f0;
+				assertTrue(tuple2.f0 < tuple2.f1);
+				assertEquals(previousMax + 1, currentMin);
+				previousMax = tuple2.f1;
+			}
+		}
+	}
+
+	@Test(expected = InvalidProgramException.class)
+	public void testRangePartitionInIteration() throws Exception {
+
+		// does not apply for collection execution
+		if (super.mode == TestExecutionMode.COLLECTION) {
+			throw new InvalidProgramException("Does not apply for collection execution");
+		}
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSource<Long> source = env.generateSequence(0, 10000);
+
+		DataSet<Tuple2<Long, String>> tuples = source.map(new MapFunction<Long, Tuple2<Long, String>>() {
+			@Override
+			public Tuple2<Long, String> map(Long v) throws Exception {
+				return new Tuple2<>(v, Long.toString(v));
+			}
+		});
+
+		DeltaIteration<Tuple2<Long, String>, Tuple2<Long, String>> it = tuples.iterateDelta(tuples, 10, 0);
+		DataSet<Tuple2<Long, String>> body = it.getWorkset()
+			.partitionByRange(1) // Verify that range partition is not allowed in iteration
+			.join(it.getSolutionSet())
+			.where(0).equalTo(0).projectFirst(0).projectSecond(1);
+		DataSet<Tuple2<Long, String>> result = it.closeWith(body, body);
+
+		result.collect(); // should fail
+	}
+
+	@Test
+	public void testRangePartitionerOnSequenceDataWithOrders() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple2<Long, Long>> dataSet = env.generateSequence(0, 10000)
+				.map(new MapFunction<Long, Tuple2<Long, Long>>() {
+			@Override
+			public Tuple2<Long, Long> map(Long value) throws Exception {
+				return new Tuple2<>(value / 5000, value % 5000);
+			}
+		});
+
+		final Tuple2Comparator<Long> tuple2Comparator = new Tuple2Comparator<>(new LongComparator(true),
+				new LongComparator(false));
+
+		MinMaxSelector<Tuple2<Long, Long>> minMaxSelector = new MinMaxSelector<>(tuple2Comparator);
+
+		final List<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>> collected = dataSet.partitionByRange(0, 1)
+				.withOrders(Order.ASCENDING, Order.DESCENDING)
+				.mapPartition(minMaxSelector)
+				.collect();
+
+		Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator));
+
+		Tuple2<Long, Long> previousMax = null;
+		for (Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> tuple2 : collected) {
+			assertTrue("Min element in each partition should be smaller than max.",
+					tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0);
+			if (previousMax == null) {
+				previousMax = tuple2.f1;
+			} else {
+				assertTrue("Partitions overlap. Previous max should be smaller than current min.",
+						tuple2Comparator.compare(previousMax, tuple2.f0) < 0);
+				if (previousMax.f0.equals(tuple2.f0.f0)) {
+					//check that ordering on the second key is correct
+					assertEquals("Ordering on the second field should be continous.",
+							previousMax.f1 - 1, tuple2.f0.f1.longValue());
+				}
+				previousMax = tuple2.f1;
+			}
+		}
+	}
+
+	@Test
+	public void testRangePartitionerOnSequenceNestedDataWithOrders() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final DataSet<Tuple2<Tuple2<Long, Long>, Long>> dataSet = env.generateSequence(0, 10000)
+				.map(new MapFunction<Long, Tuple2<Tuple2<Long, Long>, Long>>() {
+					@Override
+					public Tuple2<Tuple2<Long, Long>, Long> map(Long value) throws Exception {
+						return new Tuple2<>(new Tuple2<>(value / 5000, value % 5000), value);
+					}
+				});
+
+		final Tuple2Comparator<Long> tuple2Comparator = new Tuple2Comparator<>(new LongComparator(true),
+				new LongComparator(true));
+		MinMaxSelector<Tuple2<Long, Long>> minMaxSelector = new MinMaxSelector<>(tuple2Comparator);
+
+		final List<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>> collected = dataSet.partitionByRange(0)
+				.withOrders(Order.ASCENDING)
+				.mapPartition(new MapPartitionFunction<Tuple2<Tuple2<Long, Long>, Long>, Tuple2<Long, Long>>() {
+					@Override
+					public void mapPartition(Iterable<Tuple2<Tuple2<Long, Long>, Long>> values,
+							Collector<Tuple2<Long, Long>> out) throws Exception {
+						for (Tuple2<Tuple2<Long, Long>, Long> value : values) {
+							out.collect(value.f0);
+						}
+					}
+				})
+				.mapPartition(minMaxSelector)
+				.collect();
+
+		Collections.sort(collected, new Tuple2Comparator<>(tuple2Comparator));
+
+		Tuple2<Long, Long> previousMax = null;
+		for (Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> tuple2 : collected) {
+			assertTrue("Min element in each partition should be smaller than max.",
+					tuple2Comparator.compare(tuple2.f0, tuple2.f1) <= 0);
+			if (previousMax == null) {
+				previousMax = tuple2.f1;
+			} else {
+				assertTrue("Partitions overlap. Previous max should be smaller than current min.",
+						tuple2Comparator.compare(previousMax, tuple2.f0) < 0);
+				if (previousMax.f0.equals(tuple2.f0.f0)) {
+					assertEquals("Ordering on the second field should be continous.",
+							previousMax.f1 + 1, tuple2.f0.f1.longValue());
+				}
+				previousMax = tuple2.f1;
+			}
+		}
+	}
+
+	@Test
+	public void testRangePartitionerWithKeySelectorOnSequenceNestedDataWithOrders() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final DataSet<Tuple2<ComparablePojo, Long>> dataSet = env.generateSequence(0, 10000)
+				.map(new MapFunction<Long, Tuple2<ComparablePojo, Long>>() {
+					@Override
+					public Tuple2<ComparablePojo, Long> map(Long value) throws Exception {
+						return new Tuple2<>(new ComparablePojo(value / 5000, value % 5000), value);
+					}
+				});
+
+		final List<Tuple2<ComparablePojo, ComparablePojo>> collected = dataSet
+				.partitionByRange(new KeySelector<Tuple2<ComparablePojo, Long>, ComparablePojo>() {
+					@Override
+					public ComparablePojo getKey(Tuple2<ComparablePojo, Long> value) throws Exception {
+						return value.f0;
+					}
+				})
+				.withOrders(Order.ASCENDING)
+				.mapPartition(new MinMaxSelector<>(new ComparablePojoComparator()))
+				.mapPartition(new ExtractComparablePojo())
+				.collect();
+
+		final Comparator<Tuple2<ComparablePojo, ComparablePojo>> pojoComparator =
+				new Comparator<Tuple2<ComparablePojo, ComparablePojo>>() {
+			@Override
+			public int compare(Tuple2<ComparablePojo, ComparablePojo> o1, Tuple2<ComparablePojo, ComparablePojo> o2) {
+				return o1.f0.compareTo(o2.f1);
+			}
+		};
+		Collections.sort(collected, pojoComparator);
+
+		ComparablePojo previousMax = null;
+		for (Tuple2<ComparablePojo, ComparablePojo> element : collected) {
+			assertTrue("Min element in each partition should be smaller than max.",
+					element.f0.compareTo(element.f1) <= 0);
+			if (previousMax == null) {
+				previousMax = element.f1;
+			} else {
+				assertTrue("Partitions overlap. Previous max should be smaller than current min.",
+						previousMax.compareTo(element.f0) < 0);
+				if (previousMax.first.equals(element.f0.first)) {
+					assertEquals("Ordering on the second field should be continous.",
+							previousMax.second - 1, element.f0.second.longValue());
+				}
+				previousMax = element.f1;
+			}
+		}
+	}
+
+	private static class ExtractComparablePojo implements MapPartitionFunction<
+			Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>>,
+			Tuple2<ComparablePojo, ComparablePojo>> {
+
+		@Override
+		public void mapPartition(Iterable<Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>>> values,
+				Collector<Tuple2<ComparablePojo, ComparablePojo>> out) throws Exception {
+			for (Tuple2<Tuple2<ComparablePojo, Long>, Tuple2<ComparablePojo, Long>> value : values) {
+				out.collect(new Tuple2<>(value.f0.f0, value.f1.f0));
+			}
+		}
+	}
+
+	private static class ComparablePojoComparator implements Comparator<Tuple2<ComparablePojo, Long>>, Serializable {
+
+		@Override
+		public int compare(Tuple2<ComparablePojo, Long> o1, Tuple2<ComparablePojo, Long> o2) {
+			return o1.f0.compareTo(o2.f0);
+		}
+	}
+
+	private static class ComparablePojo implements Comparable<ComparablePojo> {
+		private Long first;
+		private Long second;
+
+		public Long getFirst() {
+			return first;
+		}
+
+		public void setFirst(Long first) {
+			this.first = first;
+		}
+
+		public Long getSecond() {
+			return second;
+		}
+
+		public void setSecond(Long second) {
+			this.second = second;
+		}
+
+		public ComparablePojo(Long first, Long second) {
+			this.first = first;
+			this.second = second;
+		}
+
+		public ComparablePojo() {
+		}
+
+		@Override
+		public int compareTo(ComparablePojo o) {
+			final int firstResult = Long.compare(this.first, o.first);
+			if (firstResult == 0) {
+				return (-1) * Long.compare(this.second, o.second);
+			}
+
+			return firstResult;
+		}
+	}
+
+	private static class ObjectSelfKeySelector implements KeySelector<Long, Long> {
+		@Override
+		public Long getKey(Long value) throws Exception {
+			return value;
+		}
+	}
+
+	private static class MinMaxSelector<T> implements MapPartitionFunction<T, Tuple2<T, T>> {
+
+		private final Comparator<T> comparator;
+
+		public MinMaxSelector(Comparator<T> comparator) {
+			this.comparator = comparator;
+		}
+
+		@Override
+		public void mapPartition(Iterable<T> values, Collector<Tuple2<T, T>> out) throws Exception {
+			Iterator<T> itr = values.iterator();
+			T min = itr.next();
+			T max = min;
+			T value;
+			while (itr.hasNext()) {
+				value = itr.next();
+				if (comparator.compare(value, min) < 0) {
+					min = value;
+				}
+				if (comparator.compare(value, max) > 0) {
+					max = value;
+				}
+
+			}
+
+			Tuple2<T, T> result = new Tuple2<>(min, max);
+			out.collect(result);
+		}
+	}
+
+	private static class Tuple2Comparator<T> implements Comparator<Tuple2<T, T>>, Serializable {
+
+		private final Comparator<T> firstComparator;
+		private final Comparator<T> secondComparator;
+
+		public Tuple2Comparator(Comparator<T> comparator) {
+			this(comparator, comparator);
+		}
+
+		public Tuple2Comparator(Comparator<T> firstComparator,
+								Comparator<T> secondComparator) {
+			this.firstComparator = firstComparator;
+			this.secondComparator = secondComparator;
+		}
+
+		@Override
+		public int compare(Tuple2<T, T> first, Tuple2<T, T> second) {
+			long result = firstComparator.compare(first.f0, second.f0);
+			if (result > 0) {
+				return 1;
+			} else if (result < 0) {
+				return -1;
+			}
+
+			result = secondComparator.compare(first.f1, second.f1);
+			if (result > 0) {
+				return 1;
+			} else if (result < 0) {
+				return -1;
+			}
+
+			return 0;
+		}
+	}
+
+	private static class LongComparator implements Comparator<Long>, Serializable {
+
+		private final boolean ascending;
+
+		public LongComparator(boolean ascending) {
+			this.ascending = ascending;
+		}
+
+		@Override
+		public int compare(Long o1, Long o2) {
+			if (ascending) {
+				return Long.compare(o1, o2);
+			} else {
+				return (-1) * Long.compare(o1, o2);
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java
new file mode 100644
index 0000000..6f52e58
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ProjectITCase.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.util.List;
+
+/**
+ * Integration tests for {@link DataSet#project}.
+ */
+public class ProjectITCase extends JavaProgramTestBase {
+
+	@Override
+	protected void testProgram() throws Exception {
+		/*
+		 * Projection with tuple fields indexes
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<String, Long, Integer>> projDs = ds.
+				project(3, 4, 2);
+		List<Tuple3<String, Long, Integer>> result = projDs.collect();
+
+		String expectedResult = "Hallo,1,0\n" +
+				"Hallo Welt,2,1\n" +
+				"Hallo Welt wie,1,2\n" +
+				"Hallo Welt wie gehts?,2,3\n" +
+				"ABC,2,4\n" +
+				"BCD,3,5\n" +
+				"CDE,2,6\n" +
+				"DEF,1,7\n" +
+				"EFG,1,8\n" +
+				"FGH,2,9\n" +
+				"GHI,1,10\n" +
+				"HIJ,3,11\n" +
+				"IJK,3,12\n" +
+				"JKL,2,13\n" +
+				"KLM,2,14\n";
+
+		compareResultAsTuples(result, expectedResult);
+	}
+
+}


[10/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java b/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java
index 6215f31..638eb5c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/OverwriteObjects.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.types.IntValue;
+
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,13 +42,13 @@ import java.util.Random;
 
 import static org.hamcrest.Matchers.is;
 
-/*
+/**
  * These programs demonstrate the effects of user defined functions which modify input objects or return locally created
  * objects that are retained and reused on future calls. The programs do not retain and later modify input objects.
  */
 public class OverwriteObjects {
 
-	public final static Logger LOG = LoggerFactory.getLogger(OverwriteObjects.class);
+	public static final Logger LOG = LoggerFactory.getLogger(OverwriteObjects.class);
 
 	// DataSets are created with this number of elements
 	private static final int NUMBER_OF_ELEMENTS = 3_000_000;
@@ -71,7 +72,7 @@ public class OverwriteObjects {
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
-		for (int parallelism = MAX_PARALLELISM ; parallelism > 0 ; parallelism--) {
+		for (int parallelism = MAX_PARALLELISM; parallelism > 0; parallelism--) {
 			LOG.info("Parallelism = {}", parallelism);
 
 			env.setParallelism(parallelism);

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java b/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java
index c8604cb..46be968 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/ReducePerformance.java
@@ -36,7 +36,7 @@ import java.util.Random;
  * (See also http://peel-framework.org/2016/04/07/hash-aggregations-in-flink.html)
  */
 public class ReducePerformance {
-	
+
 	public static void main(String[] args) throws Exception {
 
 		final int numElements = 40_000_000;
@@ -120,7 +120,7 @@ public class ReducePerformance {
 			int rem = numElements % numPartitions;
 			SplittableRandomIterator<T, B>[] res = new SplittableRandomIterator[numPartitions];
 			for (int i = 0; i < numPartitions; i++) {
-				res[i] = new SplittableRandomIterator<T, B>(i < rem ? splitSize : splitSize + 1, (B)baseIterator.copy());
+				res[i] = new SplittableRandomIterator<T, B>(i < rem ? splitSize : splitSize + 1, (B) baseIterator.copy());
 			}
 			return res;
 		}
@@ -140,7 +140,6 @@ public class ReducePerformance {
 		CopyableIterator<T> copy();
 	}
 
-
 	private static final class TupleIntIntIterator implements CopyableIterator<Tuple2<Integer, Integer>>, Serializable {
 
 		private final int keyRange;
@@ -183,7 +182,6 @@ public class ReducePerformance {
 		}
 	}
 
-
 	private static final class TupleStringIntIterator implements CopyableIterator<Tuple2<String, Integer>>, Serializable {
 
 		private final int keyRange;
@@ -226,7 +224,6 @@ public class ReducePerformance {
 		}
 	}
 
-
 	private static final class SumReducer<K> implements ReduceFunction<Tuple2<K, Integer>> {
 		@Override
 		public Tuple2<K, Integer> reduce(Tuple2<K, Integer> a, Tuple2<K, Integer> b) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
index 90dbe80..c7f43fa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java
@@ -31,24 +31,27 @@ import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 
 import static org.junit.Assert.fail;
 
+/**
+ * Manual test to evaluate impact of checkpointing on latency.
+ */
 public class StreamingScalabilityAndLatency {
-	
+
 	public static void main(String[] args) throws Exception {
 		if ((Runtime.getRuntime().maxMemory() >>> 20) < 5000) {
 			throw new RuntimeException("This test program needs to run with at least 5GB of heap space.");
 		}
-		
-		final int TASK_MANAGERS = 1;
-		final int SLOTS_PER_TASK_MANAGER = 80;
-		final int PARALLELISM = TASK_MANAGERS * SLOTS_PER_TASK_MANAGER;
+
+		final int taskManagers = 1;
+		final int slotsPerTaskManager = 80;
+		final int parallelism = taskManagers * slotsPerTaskManager;
 
 		LocalFlinkMiniCluster cluster = null;
 
 		try {
 			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, TASK_MANAGERS);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, taskManagers);
 			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 80L);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TASK_MANAGER);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slotsPerTaskManager);
 			config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 20000);
 
 			config.setInteger("taskmanager.net.server.numThreads", 1);
@@ -56,8 +59,8 @@ public class StreamingScalabilityAndLatency {
 
 			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
-			
-			runPartitioningProgram(cluster.getLeaderRPCPort(), PARALLELISM);
+
+			runPartitioningProgram(cluster.getLeaderRPCPort(), parallelism);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -69,7 +72,7 @@ public class StreamingScalabilityAndLatency {
 			}
 		}
 	}
-	
+
 	private static void runPartitioningProgram(int jobManagerPort, int parallelism) throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort);
 		env.setParallelism(parallelism);
@@ -83,23 +86,22 @@ public class StreamingScalabilityAndLatency {
 			.map(new IdMapper<Tuple2<Long, Long>>())
 			.keyBy(0)
 			.addSink(new TimestampingSink());
-		
+
 		env.execute("Partitioning Program");
 	}
-	
-	public static class TimeStampingSource implements ParallelSourceFunction<Tuple2<Long, Long>> {
+
+	private static class TimeStampingSource implements ParallelSourceFunction<Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = -151782334777482511L;
 
 		private volatile boolean running = true;
-		
-		
+
 		@Override
 		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-			
+
 			long num = 100;
 			long counter = (long) (Math.random() * 4096);
-			
+
 			while (running) {
 				if (num < 100) {
 					num++;
@@ -119,14 +121,14 @@ public class StreamingScalabilityAndLatency {
 			running = false;
 		}
 	}
-	
-	public static class TimestampingSink implements SinkFunction<Tuple2<Long, Long>> {
+
+	private static class TimestampingSink implements SinkFunction<Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1876986644706201196L;
 
 		private long maxLatency;
-		private long count; 
-		
+		private long count;
+
 		@Override
 		public void invoke(Tuple2<Long, Long> value) {
 			long ts = value.f1;
@@ -134,7 +136,7 @@ public class StreamingScalabilityAndLatency {
 				long diff = System.currentTimeMillis() - ts;
 				maxLatency = Math.max(diff, maxLatency);
 			}
-			
+
 			count++;
 			if (count == 5000) {
 				System.out.println("Max latency: " + maxLatency);
@@ -144,7 +146,7 @@ public class StreamingScalabilityAndLatency {
 		}
 	}
 
-	public static class IdMapper<T> implements MapFunction<T, T> {
+	private static class IdMapper<T> implements MapFunction<T, T> {
 
 		private static final long serialVersionUID = -6543809409233225099L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java b/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java
index 1c5744d..bd5123a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/package-info.java
@@ -21,4 +21,5 @@
  * need to be manually invoked, because they are extremely heavy, time intensive,
  * of require larger-than-usual JVMs.
  */
-package org.apache.flink.test.manual;
\ No newline at end of file
+
+package org.apache.flink.test.manual;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
index eea2509..1fb5e65 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
@@ -18,13 +18,10 @@
 
 package org.apache.flink.test.misc;
 
-import static org.junit.Assert.*;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.RichMapPartitionFunction;
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -33,6 +30,7 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -41,6 +39,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 /**
  * This test verifies that the auto parallelism is properly forwarded to the runtime.
  */
@@ -79,7 +80,6 @@ public class AutoParallelismITCase extends TestLogger {
 		}
 	}
 
-
 	@Test
 	public void testProgramWithAutoParallelism() {
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
index 39a08d2..b8f1d80 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomPartitioningITCase.java
@@ -24,8 +24,12 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.test.util.JavaProgramTestBase;
+
 import org.junit.Assert;
 
+/**
+ * Integration tests for custom {@link Partitioner}.
+ */
 @SuppressWarnings("serial")
 public class CustomPartitioningITCase extends JavaProgramTestBase {
 
@@ -36,17 +40,17 @@ public class CustomPartitioningITCase extends JavaProgramTestBase {
 		if (!isCollectionExecution()) {
 			Assert.assertTrue(env.getParallelism() > 1);
 		}
-		
+
 		env.generateSequence(1, 1000)
 			.partitionCustom(new AllZeroPartitioner(), new IdKeySelector<Long>())
 			.map(new FailExceptInPartitionZeroMapper())
 			.output(new DiscardingOutputFormat<Long>());
-		
+
 		env.execute();
 	}
-	
-	public static class FailExceptInPartitionZeroMapper extends RichMapFunction<Long, Long> {
-		
+
+	private static class FailExceptInPartitionZeroMapper extends RichMapFunction<Long, Long> {
+
 		@Override
 		public Long map(Long value) throws Exception {
 			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
@@ -56,15 +60,15 @@ public class CustomPartitioningITCase extends JavaProgramTestBase {
 			}
 		}
 	}
-	
-	public static class AllZeroPartitioner implements Partitioner<Long> {
+
+	private static class AllZeroPartitioner implements Partitioner<Long> {
 		@Override
 		public int partition(Long key, int numPartitions) {
 			return 0;
 		}
 	}
-	
-	public static class IdKeySelector<T> implements KeySelector<T, T> {
+
+	private static class IdKeySelector<T> implements KeySelector<T, T> {
 		@Override
 		public T getKey(T value) {
 			return value;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
index 76480ba..1532741 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
@@ -30,8 +30,8 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.types.Value;
-
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -41,11 +41,15 @@ import java.io.IOException;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test for proper error messages in case user-defined serialization is broken
+ * and detected in the network stack.
+ */
 @SuppressWarnings("serial")
 public class CustomSerializationITCase extends TestLogger {
 
 	private static final int PARLLELISM = 5;
-	
+
 	private static LocalFlinkMiniCluster cluster;
 
 	private static TestEnvironment env;
@@ -66,13 +70,13 @@ public class CustomSerializationITCase extends TestLogger {
 		cluster.shutdown();
 		cluster = null;
 	}
-	
+
 	@Test
 	public void testIncorrectSerializer1() {
 		try {
 			env.setParallelism(PARLLELISM);
 			env.getConfig().disableSysoutLogging();
-			
+
 			env
 				.generateSequence(1, 10 * PARLLELISM)
 				.map(new MapFunction<Long, ConsumesTooMuch>() {
@@ -83,7 +87,7 @@ public class CustomSerializationITCase extends TestLogger {
 				})
 				.rebalance()
 				.output(new DiscardingOutputFormat<ConsumesTooMuch>());
-			
+
 			env.execute();
 		}
 		catch (JobExecutionException e) {
@@ -186,11 +190,14 @@ public class CustomSerializationITCase extends TestLogger {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Custom Data Types with broken Serialization Logic
 	// ------------------------------------------------------------------------
-	
+
+	/**
+	 * {@link Value} reading more data than written.
+	 */
 	public static class ConsumesTooMuch implements Value {
 
 		@Override
@@ -206,6 +213,9 @@ public class CustomSerializationITCase extends TestLogger {
 		}
 	}
 
+	/**
+	 * {@link Value} reading more buffers than written.
+	 */
 	public static class ConsumesTooMuchSpanning implements Value {
 
 		@Override
@@ -221,6 +231,9 @@ public class CustomSerializationITCase extends TestLogger {
 		}
 	}
 
+	/**
+	 * {@link Value} reading less data than written.
+	 */
 	public static class ConsumesTooLittle implements Value {
 
 		@Override
@@ -236,6 +249,9 @@ public class CustomSerializationITCase extends TestLogger {
 		}
 	}
 
+	/**
+	 * {@link Value} reading fewer buffers than written.
+	 */
 	public static class ConsumesTooLittleSpanning implements Value {
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java b/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java
index fa1fcb6..c004759 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/GenericTypeInfoTest.java
@@ -21,20 +21,23 @@ package org.apache.flink.test.misc;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+/**
+ * Test TypeInfo serializer tree.
+ */
 public class GenericTypeInfoTest {
 
 	@Test
 	public void testSerializerTree() {
 		@SuppressWarnings("unchecked")
-		TypeInformation<CollectionDataSets.PojoWithCollectionGeneric> ti = 
-				(TypeInformation<CollectionDataSets.PojoWithCollectionGeneric>) 
+		TypeInformation<CollectionDataSets.PojoWithCollectionGeneric> ti =
+				(TypeInformation<CollectionDataSets.PojoWithCollectionGeneric>)
 						TypeExtractor.createTypeInfo(CollectionDataSets.PojoWithCollectionGeneric.class);
-		
+
 		String serTree = Utils.getSerializerTree(ti);
 		// We can not test against the entire output because the fields of 'String' differ
 		// between java versions
@@ -67,7 +70,7 @@ public class GenericTypeInfoTest {
 				"            lowestSetBit:int\n" +
 				"            firstNonzeroIntNum:int\n" +
 				"    mixed:java.util.List\n" +
-				"    makeMeGeneric:org.apache.flink.test.javaApiOperators.util.CollectionDataSets$PojoWithDateAndEnum\n" +
+				"    makeMeGeneric:org.apache.flink.test.operators.util.CollectionDataSets$PojoWithDateAndEnum\n" +
 				"        group:java.lang.String\n"));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
index 7dab0f1..00b4485 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
@@ -30,24 +30,26 @@ import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
-
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for the system behavior in multiple corner cases
  *   - when null records are passed through the system.
  *   - when disjoint dataflows are executed
  *   - when accumulators are used chained after a non-udf operator.
- *   
- * The tests are bundled into one class to reuse the same test cluster. This speeds
+ *
+ * <p>The tests are bundled into one class to reuse the same test cluster. This speeds
  * up test execution, as the majority of the test time goes usually into starting/stopping the
  * test cluster.
  */
@@ -59,7 +61,7 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 	private static LocalFlinkMiniCluster cluster;
 
 	private static TestEnvironment env;
-	
+
 	@BeforeClass
 	public static void startCluster() {
 		Configuration config = new Configuration();
@@ -72,13 +74,13 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 
 		env = new TestEnvironment(cluster, PARALLELISM, false);
 	}
-	
+
 	@AfterClass
 	public static void shutdownCluster() {
 		cluster.shutdown();
 		cluster = null;
 	}
-	
+
 	@Test
 	public void testNullValues() {
 		try {
@@ -128,13 +130,13 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 
 	@Test
 	public void testAccumulatorsAfterNoOp() {
-		
-		final String ACC_NAME = "test_accumulator";
-		
+
+		final String accName = "test_accumulator";
+
 		try {
 			env.setParallelism(6);
 			env.getConfig().disableSysoutLogging();
-			
+
 			env.generateSequence(1, 1000000)
 					.rebalance()
 					.flatMap(new RichFlatMapFunction<Long, Long>() {
@@ -143,7 +145,7 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 
 						@Override
 						public void open(Configuration parameters) {
-							counter = getRuntimeContext().getLongCounter(ACC_NAME);
+							counter = getRuntimeContext().getLongCounter(accName);
 						}
 
 						@Override
@@ -154,8 +156,8 @@ public class MiscellaneousIssuesITCase extends TestLogger {
 					.output(new DiscardingOutputFormat<Long>());
 
 			JobExecutionResult result = env.execute();
-			
-			assertEquals(1000000L, result.getAllAccumulatorResults().get(ACC_NAME));
+
+			assertEquals(1000000L, result.getAllAccumulatorResults().get(accName));
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
index a5103cc..fd556d5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
@@ -32,35 +32,41 @@ import org.apache.flink.examples.java.clustering.KMeans;
 import org.apache.flink.examples.java.clustering.util.KMeansData;
 import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.examples.java.graph.util.ConnectedComponentsData;
-
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+/**
+ * Test that runs an iterative job after a failure in another iterative job.
+ * This test validates that task slots in co-location constraints are properly
+ * freed in the presence of failures.
+ */
 public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 
 	private static final int PARALLELISM = 16;
 	@Test
 	public void testSuccessfulProgramAfterFailure() {
 		LocalFlinkMiniCluster cluster = null;
-		
+
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 80L);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
 			config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 800);
-			
+
 			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 
 			TestEnvironment env = new TestEnvironment(cluster, PARALLELISM, false);
-			
+
 			try {
 				runConnectedComponents(env);
 			}
@@ -68,7 +74,7 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 				e.printStackTrace();
 				fail("Program Execution should have succeeded.");
 			}
-	
+
 			try {
 				runKMeans(env);
 				fail("This program execution should have failed.");
@@ -76,7 +82,7 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 			catch (JobExecutionException e) {
 				assertTrue(e.getCause().getMessage().contains("Insufficient number of network buffers"));
 			}
-	
+
 			try {
 				runConnectedComponents(env);
 			}
@@ -95,9 +101,9 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 			}
 		}
 	}
-	
+
 	private static void runConnectedComponents(ExecutionEnvironment env) throws Exception {
-		
+
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
@@ -166,7 +172,7 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger {
 				.map(new KMeans.SelectNearestCenter()).withBroadcastSet(finalCentroids, "centroids");
 
 		clusteredPoints.output(new DiscardingOutputFormat<Tuple2<Integer, KMeans.Point>>());
-		
+
 		env.execute("KMeans Example");
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java
new file mode 100644
index 0000000..b4bd213
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/AggregateITCase.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.ValueCollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.StringValue;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Integration tests for aggregations.
+ */
+@RunWith(Parameterized.class)
+public class AggregateITCase extends MultipleProgramsTestBase {
+
+	public AggregateITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testFullAggregate() throws Exception {
+		/*
+		 * Full Aggregate
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> aggregateDs = ds
+				.aggregate(Aggregations.SUM, 0)
+				.and(Aggregations.MAX, 1)
+				.project(0, 1);
+
+		List<Tuple2<Integer, Long>> result = aggregateDs.collect();
+
+		String expected = "231,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testFullAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Full Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds
+				.aggregate(Aggregations.SUM, 0)
+				.and(Aggregations.MAX, 1)
+				.project(0, 1);
+
+		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
+
+		String expected = "231,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testGroupedAggregate() throws Exception {
+		/*
+		 * Grouped Aggregate
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Long, Integer>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.SUM, 0)
+				.project(1, 0);
+
+		List<Tuple2<Long, Integer>> result = aggregateDs.collect();
+
+		String expected = "1,1\n" +
+				"2,5\n" +
+				"3,15\n" +
+				"4,34\n" +
+				"5,65\n" +
+				"6,111\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testGroupedAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Grouped Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.SUM, 0)
+				.project(1, 0);
+
+		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
+
+		String expected = "1,1\n" +
+				"2,5\n" +
+				"3,15\n" +
+				"4,34\n" +
+				"5,65\n" +
+				"6,111\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedAggregate() throws Exception {
+		/*
+		 * Nested Aggregate
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple1<Integer>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.MIN, 0)
+				.aggregate(Aggregations.MIN, 0)
+				.project(0);
+
+		List<Tuple1<Integer>> result = aggregateDs.collect();
+
+		String expected = "1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedAggregateOfMutableValueTypes() throws Exception {
+		/*
+		 * Nested Aggregate of mutable value types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple1<IntValue>> aggregateDs = ds.groupBy(1)
+				.aggregate(Aggregations.MIN, 0)
+				.aggregate(Aggregations.MIN, 0)
+				.project(0);
+
+		List<Tuple1<IntValue>> result = aggregateDs.collect();
+
+		String expected = "1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java
new file mode 100644
index 0000000..4108b24
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupGroupSortITCase.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.types.NullValue;
+import org.apache.flink.util.Collector;
+
+import org.junit.Assert;
+
+/**
+ * Integration tests for {@link CoGroupFunction}.
+ */
+@SuppressWarnings({"serial", "unchecked"})
+public class CoGroupGroupSortITCase extends JavaProgramTestBase {
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Long, Long>> input1 = env.fromElements(
+				new Tuple2<Long, Long>(0L, 5L),
+				new Tuple2<Long, Long>(0L, 4L),
+				new Tuple2<Long, Long>(0L, 3L),
+				new Tuple2<Long, Long>(0L, 2L),
+				new Tuple2<Long, Long>(0L, 1L),
+				new Tuple2<Long, Long>(1L, 10L),
+				new Tuple2<Long, Long>(1L, 8L),
+				new Tuple2<Long, Long>(1L, 9L),
+				new Tuple2<Long, Long>(1L, 7L));
+
+		DataSet<TestPojo> input2 = env.fromElements(
+				new TestPojo(0L, 10L, 3L),
+				new TestPojo(0L, 8L, 3L),
+				new TestPojo(0L, 10L, 1L),
+				new TestPojo(0L, 9L, 0L),
+				new TestPojo(0L, 8L, 2L),
+				new TestPojo(0L, 8L, 4L),
+				new TestPojo(1L, 10L, 3L),
+				new TestPojo(1L, 8L, 3L),
+				new TestPojo(1L, 10L, 1L),
+				new TestPojo(1L, 9L, 0L),
+				new TestPojo(1L, 8L, 2L),
+				new TestPojo(1L, 8L, 4L));
+
+		input1.coGroup(input2)
+		.where(1).equalTo("b")
+		.sortFirstGroup(0, Order.DESCENDING)
+		.sortSecondGroup("c", Order.ASCENDING).sortSecondGroup("a", Order.DESCENDING)
+
+		.with(new ValidatingCoGroup())
+		.output(new DiscardingOutputFormat<NullValue>());
+
+		env.execute();
+	}
+
+	private static class ValidatingCoGroup implements CoGroupFunction<Tuple2<Long, Long>, TestPojo, NullValue> {
+
+		@Override
+		public void coGroup(Iterable<Tuple2<Long, Long>> first, Iterable<TestPojo> second, Collector<NullValue> out) throws Exception {
+			// validate the tuple input, field 1, descending
+			{
+				long lastValue = Long.MAX_VALUE;
+
+				for (Tuple2<Long, Long> t : first) {
+					long current = t.f1;
+					Assert.assertTrue(current <= lastValue);
+					lastValue = current;
+				}
+			}
+
+			// validate the pojo input
+			{
+				TestPojo lastValue = new TestPojo(Long.MAX_VALUE, 0, Long.MIN_VALUE);
+
+				for (TestPojo current : second) {
+					Assert.assertTrue(current.c >= lastValue.c);
+					Assert.assertTrue(current.c != lastValue.c || current.a <= lastValue.a);
+
+					lastValue = current;
+				}
+			}
+
+		}
+	}
+
+	/**
+	 * Test POJO.
+	 */
+	public static class TestPojo implements Cloneable {
+		public long a;
+		public long b;
+		public long c;
+
+		public TestPojo() {}
+
+		public TestPojo(long a, long b, long c) {
+			this.a = a;
+			this.b = b;
+			this.c = c;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
new file mode 100644
index 0000000..453f525
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CoGroupITCase.java
@@ -0,0 +1,989 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.distributions.DataDistribution;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.RichCoGroupFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.utils.DataSetUtils;
+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.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link CoGroupFunction} and {@link RichCoGroupFunction}.
+ */
+@RunWith(Parameterized.class)
+public class CoGroupITCase extends MultipleProgramsTestBase {
+
+	public CoGroupITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	/*
+	 * CoGroup on tuples with key field selector
+	 */
+	@Test
+	public void testCoGroupTuplesWithKeyFieldSelector() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<Integer, Integer>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5CoGroup());
+
+		List<Tuple2<Integer, Integer>> result = coGroupDs.collect();
+
+		String expected = "1,0\n" +
+				"2,6\n" +
+				"3,24\n" +
+				"4,60\n" +
+				"5,120\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupOnTwoCustomTypeInputsWithKeyExtractors() throws Exception {
+		/*
+		 * CoGroup on two custom type inputs with key extractors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> coGroupDs = ds.coGroup(ds2).where(new KeySelector4()).equalTo(new
+				KeySelector5()).with(new CustomTypeCoGroup());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = "1,0,test\n" +
+				"2,6,test\n" +
+				"3,24,test\n" +
+				"4,60,test\n" +
+				"5,120,test\n" +
+				"6,210,test\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector4 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	private static class KeySelector5 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfCoGroupIfUDFReturnsLeftInputObjects() throws Exception {
+		/*
+		 * check correctness of cogroup if UDF returns left input objects
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple3ReturnLeft());
+
+		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCoGroupIfUDFReturnsRightInputObjects() throws Exception {
+		/*
+		 * check correctness of cogroup if UDF returns right input objects
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5ReturnRight());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = coGroupDs.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"3,4,3,Hallo Welt wie gehts?,2\n" +
+				"3,5,4,ABC,2\n" +
+				"3,6,5,BCD,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithBroadcastSet() throws Exception {
+		/*
+		 * Reduce with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Integer, Integer>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5CoGroupBC()).withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<Integer, Integer, Integer>> result = coGroupDs.collect();
+
+		String expected = "1,0,55\n" +
+				"2,6,55\n" +
+				"3,24,55\n" +
+				"4,60,55\n" +
+				"5,120,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
+			throws Exception {
+		/*
+		 * CoGroup on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds.coGroup(ds2).where(2).equalTo(new
+				KeySelector2()).with(new MixedCoGroup());
+
+		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
+
+		String expected = "0,1,test\n" +
+				"1,2,test\n" +
+				"2,5,test\n" +
+				"3,15,test\n" +
+				"4,33,test\n" +
+				"5,63,test\n" +
+				"6,109,test\n" +
+				"7,4,test\n" +
+				"8,4,test\n" +
+				"9,4,test\n" +
+				"10,5,test\n" +
+				"11,5,test\n" +
+				"12,5,test\n" +
+				"13,5,test\n" +
+				"14,5,test\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCoGroupOnACustomTypeWithKeyExtractorAndATupleInputWithKeyFieldSelector()
+			throws Exception {
+		/*
+		 * CoGroup on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> coGroupDs = ds2.coGroup(ds).where(new KeySelector3()).equalTo(2).with
+				(new MixedCoGroup2());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = "0,1,test\n" +
+				"1,2,test\n" +
+				"2,5,test\n" +
+				"3,15,test\n" +
+				"4,33,test\n" +
+				"5,63,test\n" +
+				"6,109,test\n" +
+				"7,4,test\n" +
+				"8,4,test\n" +
+				"9,4,test\n" +
+				"10,5,test\n" +
+				"11,5,test\n" +
+				"12,5,test\n" +
+				"13,5,test\n" +
+				"14,5,test\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector3 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCoGroupWithMultipleKeyFieldsWithFieldSelector() throws Exception {
+		/*
+		 * CoGroup with multiple key fields
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
+				where(0, 4).equalTo(0, 1).with(new Tuple5Tuple3CoGroup());
+
+		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
+
+		String expected = "1,1,Hallo\n" +
+				"2,2,Hallo Welt\n" +
+				"3,2,Hallo Welt wie gehts?\n" +
+				"3,2,ABC\n" +
+				"5,3,HIJ\n" +
+				"5,3,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithMultipleKeyFieldsWithStaticClassKeyExtractor() throws Exception {
+		/*
+		 * CoGroup with multiple key fields
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
+				where(new KeySelector7()).
+				equalTo(new KeySelector8()).with(new Tuple5Tuple3CoGroup());
+
+		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
+
+		String expected = "1,1,Hallo\n" +
+				"2,2,Hallo Welt\n" +
+				"3,2,Hallo Welt wie gehts?\n" +
+				"3,2,ABC\n" +
+				"5,3,HIJ\n" +
+				"5,3,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithClosureCleaner() throws Exception {
+		/*
+		 * CoGroup with multiple key fields, test working closure cleaner for inner classes
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
+				where(new KeySelector<Tuple5<Integer, Long, Integer, String, Long>,
+						Tuple2<Integer, Long>>() {
+					@Override
+					public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) throws Exception {
+						return new Tuple2<Integer, Long>(t.f0, t.f4);
+					}
+				}).
+				equalTo(new KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>>() {
+
+					@Override
+					public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+						return new Tuple2<>(t.f0, t.f1);
+					}
+				}).
+				with(new CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>() {
+					@Override
+					public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+							Iterable<Tuple3<Integer, Long, String>> second,
+							Collector<Tuple3<Integer, Long, String>> out) {
+						List<String> strs = new ArrayList<>();
+
+						for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
+							strs.add(t.f3);
+						}
+
+						for (Tuple3<Integer, Long, String> t : second) {
+							for (String s : strs) {
+								out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
+							}
+						}
+					}
+				});
+
+		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
+
+		String expected = "1,1,Hallo\n" +
+				"2,2,Hallo Welt\n" +
+				"3,2,Hallo Welt wie gehts?\n" +
+				"3,2,ABC\n" +
+				"5,3,HIJ\n" +
+				"5,3,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithoutClosureCleaner() throws Exception {
+		/*
+		 * CoGroup with multiple key fields, test that disabling closure cleaner leads to an exception when using inner
+		 * classes.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().disableClosureCleaner();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+		boolean correctExceptionTriggered = false;
+		try {
+			DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
+					where(new KeySelector<Tuple5<Integer, Long, Integer, String, Long>,
+							Tuple2<Integer, Long>>() {
+						@Override
+						public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) throws Exception {
+							return new Tuple2<Integer, Long>(t.f0, t.f4);
+						}
+					}).
+					equalTo(new KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>>() {
+
+						@Override
+						public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+							return new Tuple2<Integer, Long>(t.f0, t.f1);
+						}
+					}).
+					with(new CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>() {
+						@Override
+						public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+									Iterable<Tuple3<Integer, Long, String>> second,
+									Collector<Tuple3<Integer, Long, String>> out) {
+							List<String> strs = new ArrayList<String>();
+
+							for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
+								strs.add(t.f3);
+							}
+
+							for (Tuple3<Integer, Long, String> t : second) {
+								for (String s : strs) {
+									out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
+								}
+							}
+						}
+					});
+		} catch (InvalidProgramException ex) {
+			correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException);
+		}
+		Assert.assertTrue(correctExceptionTriggered);
+
+	}
+
+	private static class KeySelector7 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>,
+	Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f4);
+		}
+	}
+
+	private static class KeySelector8 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f1);
+		}
+	}
+
+	@Test
+	public void testCoGroupTwoCustomTypeInputsWithExpressionKeys() throws Exception {
+		/*
+		 * CoGroup on two custom type inputs using expression keys
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> coGroupDs = ds.coGroup(ds2).where("myInt").equalTo("myInt").with(new CustomTypeCoGroup());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = "1,0,test\n" +
+				"2,6,test\n" +
+				"3,24,test\n" +
+				"4,60,test\n" +
+				"5,120,test\n" +
+				"6,210,test\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCoGroupOnTwoCustomTypeInputsWithExpressionKeyAndFieldSelector() throws
+			Exception {
+		/*
+		 * CoGroup on two custom type inputs using expression keys
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
+				.where("nestedPojo.longNumber").equalTo(6).with(new CoGroup1());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = 	"-1,20000,Flink\n" +
+				"-1,10000,Flink\n" +
+				"-1,30000,Flink\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class CoGroup1 implements CoGroupFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<POJO> first,
+				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
+				Collector<CustomType> out) throws Exception {
+			for (POJO p : first) {
+				for (Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
+					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
+					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
+				}
+			}
+		}
+	}
+
+	@Test
+	public void testCoGroupFieldSelectorAndComplicatedKeySelector() throws Exception {
+		/*
+		 * CoGroup field-selector (expression keys) + key selector function
+		 * The key selector is unnecessary complicated (Tuple1) ;)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
+				.where(new KeySelector6()).equalTo(6).with(new CoGroup3());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = 	"-1,20000,Flink\n" +
+				"-1,10000,Flink\n" +
+				"-1,30000,Flink\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector6 implements KeySelector<POJO, Tuple1<Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple1<Long> getKey(POJO value)
+		throws Exception {
+			return new Tuple1<Long>(value.nestedPojo.longNumber);
+		}
+	}
+
+	private static class CoGroup3 implements CoGroupFunction<POJO, Tuple7<Integer,
+			String, Integer, Integer, Long, String, Long>, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<POJO> first,
+				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
+				Collector<CustomType> out) throws Exception {
+			for (POJO p : first) {
+				for (Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
+					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
+					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
+				}
+			}
+		}
+	}
+
+	@Test
+	public void testCoGroupFieldSelectorAndKeySelector() throws Exception {
+		/*
+		 * CoGroup field-selector (expression keys) + key selector function
+		 * The key selector is simple here
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
+				.where(new KeySelector1()).equalTo(6).with(new CoGroup2());
+
+		List<CustomType> result = coGroupDs.collect();
+
+		String expected = "-1,20000,Flink\n" +
+				"-1,10000,Flink\n" +
+				"-1,30000,Flink\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithAtomicType1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Integer> ds2 = env.fromElements(0, 1, 2);
+
+		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds1.coGroup(ds2).where(0).equalTo("*").with(new CoGroupAtomic1());
+
+		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
+
+		String expected = "(1,1,Hi)\n" +
+			"(2,2,Hello)";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithAtomicType2() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Integer> ds1 = env.fromElements(0, 1, 2);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds1.coGroup(ds2).where("*").equalTo(0).with(new CoGroupAtomic2());
+
+		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
+
+		String expected = "(1,1,Hi)\n" +
+			"(2,2,Hello)";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCoGroupWithRangePartitioning() throws Exception {
+		/*
+		 * Test coGroup on tuples with multiple key field positions and same customized distribution
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+
+		env.setParallelism(4);
+		TestDistribution testDis = new TestDistribution();
+		DataSet<Tuple3<Integer, Long, String>> coGrouped =
+				DataSetUtils.partitionByRange(ds1, testDis, 0, 4)
+						.coGroup(DataSetUtils.partitionByRange(ds2, testDis, 0, 1))
+						.where(0, 4)
+						.equalTo(0, 1)
+						.with(new Tuple5Tuple3CoGroup());
+
+		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
+
+		String expected = "1,1,Hallo\n" +
+				"2,2,Hallo Welt\n" +
+				"3,2,Hallo Welt wie gehts?\n" +
+				"3,2,ABC\n" +
+				"5,3,HIJ\n" +
+				"5,3,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  UDF classes
+	// --------------------------------------------------------------------------------------------
+
+	private static class KeySelector1 implements KeySelector<POJO, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long getKey(POJO value)
+		throws Exception {
+			return value.nestedPojo.longNumber;
+		}
+	}
+
+	private static class CoGroup2 implements CoGroupFunction<POJO, Tuple7<Integer, String,
+			Integer, Integer, Long, String, Long>, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<POJO> first,
+				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
+				Collector<CustomType> out) throws Exception {
+			for (POJO p : first) {
+				for (Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
+					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
+					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
+				}
+			}
+		}
+	}
+
+	private static class Tuple5CoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Integer>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
+				Collector<Tuple2<Integer, Integer>> out) {
+			int sum = 0;
+			int id = 0;
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : first) {
+				sum += element.f2;
+				id = element.f0;
+			}
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
+				sum += element.f2;
+				id = element.f0;
+			}
+
+			out.collect(new Tuple2<Integer, Integer>(id, sum));
+		}
+	}
+
+	private static class CustomTypeCoGroup implements CoGroupFunction<CustomType, CustomType, CustomType> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<CustomType> first, Iterable<CustomType> second, Collector<CustomType> out) {
+
+			CustomType o = new CustomType(0, 0, "test");
+
+			for (CustomType element : first) {
+				o.myInt = element.myInt;
+				o.myLong += element.myLong;
+			}
+
+			for (CustomType element : second) {
+				o.myInt = element.myInt;
+				o.myLong += element.myLong;
+			}
+
+			out.collect(o);
+		}
+	}
+
+	private static class MixedCoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, CustomType, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+				Iterable<CustomType> second,
+				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+
+			long sum = 0;
+			int id = 0;
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : first) {
+				sum += element.f0;
+				id = element.f2;
+			}
+
+			for (CustomType element : second) {
+				id = element.myInt;
+				sum += element.myLong;
+			}
+
+			out.collect(new Tuple3<Integer, Long, String>(id, sum, "test"));
+		}
+
+	}
+
+	private static class MixedCoGroup2 implements CoGroupFunction<CustomType, Tuple5<Integer, Long, Integer, String, Long>, CustomType> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<CustomType> first,
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
+				Collector<CustomType> out) {
+			CustomType o = new CustomType(0, 0, "test");
+
+			for (CustomType element : first) {
+				o.myInt = element.myInt;
+				o.myLong += element.myLong;
+			}
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
+				o.myInt = element.f2;
+				o.myLong += element.f0;
+			}
+
+			out.collect(o);
+
+		}
+
+	}
+
+	private static class Tuple3ReturnLeft implements CoGroupFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<Tuple3<Integer, Long, String>> first,
+				Iterable<Tuple3<Integer, Long, String>> second,
+				Collector<Tuple3<Integer, Long, String>> out) {
+			for (Tuple3<Integer, Long, String> element : first) {
+				if (element.f0 < 6) {
+					out.collect(element);
+				}
+			}
+		}
+	}
+
+	private static class Tuple5ReturnRight implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
+				Collector<Tuple5<Integer, Long, Integer, String, Long>> out) {
+			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
+				if (element.f0 < 4) {
+					out.collect(element);
+				}
+			}
+		}
+	}
+
+	private static class Tuple5CoGroupBC extends RichCoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Integer, Integer>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private int broadcast = 42;
+
+		@Override
+		public void open(Configuration config) {
+
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			broadcast = sum;
+
+		}
+
+		@Override
+		public void coGroup(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
+				Collector<Tuple3<Integer, Integer, Integer>> out) {
+			int sum = 0;
+			int id = 0;
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : first) {
+				sum += element.f2;
+				id = element.f0;
+			}
+
+			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
+				sum += element.f2;
+				id = element.f0;
+			}
+
+			out.collect(new Tuple3<Integer, Integer, Integer>(id, sum, broadcast));
+		}
+	}
+
+	private static class Tuple5Tuple3CoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
+				Iterable<Tuple3<Integer, Long, String>> second,
+				Collector<Tuple3<Integer, Long, String>> out) {
+			List<String> strs = new ArrayList<String>();
+
+			for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
+				strs.add(t.f3);
+			}
+
+			for (Tuple3<Integer, Long, String> t : second) {
+				for (String s : strs) {
+					out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
+				}
+			}
+		}
+	}
+
+	private static class CoGroupAtomic1 implements CoGroupFunction<Tuple3<Integer, Long, String>, Integer, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<Tuple3<Integer, Long, String>> first, Iterable<Integer> second, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			List<Integer> ints = new ArrayList<Integer>();
+
+			for (Integer i : second) {
+				ints.add(i);
+			}
+
+			for (Tuple3<Integer, Long, String> t : first) {
+				for (Integer i : ints) {
+					if (t.f0.equals(i)) {
+						out.collect(t);
+					}
+				}
+			}
+		}
+	}
+
+	private static class CoGroupAtomic2 implements CoGroupFunction<Integer, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<Integer> first, Iterable<Tuple3<Integer, Long, String>> second, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			List<Integer> ints = new ArrayList<Integer>();
+
+			for (Integer i : first) {
+				ints.add(i);
+			}
+
+			for (Tuple3<Integer, Long, String> t : second) {
+				for (Integer i : ints) {
+					if (t.f0.equals(i)) {
+						out.collect(t);
+					}
+				}
+			}
+		}
+	}
+
+	/**
+	 * Test {@link DataDistribution}.
+	 */
+	public static class TestDistribution implements DataDistribution {
+		public Object[][] boundaries = new Object[][]{
+				new Object[]{2, 2L},
+				new Object[]{5, 4L},
+				new Object[]{10, 12L},
+				new Object[]{21, 6L}
+		};
+
+		public TestDistribution() {}
+
+		@Override
+		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
+			return boundaries[bucketNum];
+		}
+
+		@Override
+		public int getNumberOfFields() {
+			return 2;
+		}
+
+		@Override
+		public TypeInformation[] getKeyTypes() {
+			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO};
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj instanceof TestDistribution;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
new file mode 100644
index 0000000..6e61f60
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java
@@ -0,0 +1,457 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.CrossFunction;
+import org.apache.flink.api.common.functions.RichCrossFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link CrossFunction} and {@link RichCrossFunction}.
+ */
+@RunWith(Parameterized.class)
+public class CrossITCase extends MultipleProgramsTestBase {
+
+	public CrossITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception {
+		/*
+		 * check correctness of cross on two tuple inputs
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<Integer, String>> crossDs = ds.cross(ds2).with(new Tuple5Cross());
+
+		List<Tuple2<Integer, String>> result = crossDs.collect();
+
+		String expected = "0,HalloHallo\n" +
+				"1,HalloHallo Welt\n" +
+				"2,HalloHallo Welt wie\n" +
+				"1,Hallo WeltHallo\n" +
+				"2,Hallo WeltHallo Welt\n" +
+				"3,Hallo WeltHallo Welt wie\n" +
+				"2,Hallo Welt wieHallo\n" +
+				"3,Hallo Welt wieHallo Welt\n" +
+				"4,Hallo Welt wieHallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossIfUDFReturnsLeftInputObject() throws Exception {
+		/*
+		 * check correctness of cross if UDF returns left input object
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> crossDs = ds.cross(ds2).with(new Tuple3ReturnLeft());
+
+		List<Tuple3<Integer, Long, String>> result = crossDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"1,1,Hi\n" +
+				"1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"2,2,Hello\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"3,2,Hello world\n" +
+				"3,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossIfUDFReturnsRightInputObject() throws Exception {
+		/*
+		 * check correctness of cross if UDF returns right input object
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> crossDs = ds.cross(ds2).with(new Tuple5ReturnRight());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = crossDs
+				.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"1,1,0,Hallo,1\n" +
+				"1,1,0,Hallo,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"2,3,2,Hallo Welt wie,1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossWithBroadcastSet() throws Exception {
+		/*
+		 * check correctness of cross with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Integer, Integer>> crossDs = ds.cross(ds2).with(new Tuple5CrossBC()).withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<Integer, Integer, Integer>> result = crossDs.collect();
+
+		String expected = "2,0,55\n" +
+				"3,0,55\n" +
+				"3,0,55\n" +
+				"3,0,55\n" +
+				"4,1,55\n" +
+				"4,2,55\n" +
+				"3,0,55\n" +
+				"4,2,55\n" +
+				"4,4,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossWithHuge() throws Exception {
+		/*
+		 * check correctness of crossWithHuge (only correctness of result -> should be the same as with normal cross)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<Integer, String>> crossDs = ds.crossWithHuge(ds2).with(new Tuple5Cross());
+
+		List<Tuple2<Integer, String>> result = crossDs.collect();
+
+		String expected = "0,HalloHallo\n" +
+				"1,HalloHallo Welt\n" +
+				"2,HalloHallo Welt wie\n" +
+				"1,Hallo WeltHallo\n" +
+				"2,Hallo WeltHallo Welt\n" +
+				"3,Hallo WeltHallo Welt wie\n" +
+				"2,Hallo Welt wieHallo\n" +
+				"3,Hallo Welt wieHallo Welt\n" +
+				"4,Hallo Welt wieHallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossWithTiny() throws Exception {
+		/*
+		 * check correctness of crossWithTiny (only correctness of result -> should be the same as with normal cross)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<Integer, String>> crossDs = ds.crossWithTiny(ds2).with(new Tuple5Cross());
+
+		List<Tuple2<Integer, String>> result = crossDs.collect();
+
+		String expected = "0,HalloHallo\n" +
+				"1,HalloHallo Welt\n" +
+				"2,HalloHallo Welt wie\n" +
+				"1,Hallo WeltHallo\n" +
+				"2,Hallo WeltHallo Welt\n" +
+				"3,Hallo WeltHallo Welt wie\n" +
+				"2,Hallo Welt wieHallo\n" +
+				"3,Hallo Welt wieHallo Welt\n" +
+				"4,Hallo Welt wieHallo Welt wie\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testProjectCrossOnATupleInput1() throws Exception{
+		/*
+		 * project cross on a tuple input 1
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple6<String, Long, String, Integer, Long, Long>> crossDs = ds.cross(ds2)
+				.projectFirst(2, 1)
+				.projectSecond(3)
+				.projectFirst(0)
+				.projectSecond(4, 1);
+
+		List<Tuple6<String, Long, String, Integer, Long, Long>> result = crossDs.collect();
+
+		String expected = "Hi,1,Hallo,1,1,1\n" +
+				"Hi,1,Hallo Welt,1,2,2\n" +
+				"Hi,1,Hallo Welt wie,1,1,3\n" +
+				"Hello,2,Hallo,2,1,1\n" +
+				"Hello,2,Hallo Welt,2,2,2\n" +
+				"Hello,2,Hallo Welt wie,2,1,3\n" +
+				"Hello world,2,Hallo,3,1,1\n" +
+				"Hello world,2,Hallo Welt,3,2,2\n" +
+				"Hello world,2,Hallo Welt wie,3,1,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testProjectCrossOnATupleInput2() throws Exception {
+		/*
+		 * project cross on a tuple input 2
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple6<String, String, Long, Long, Long, Integer>> crossDs = ds.cross(ds2)
+				.projectSecond(3)
+				.projectFirst(2, 1)
+				.projectSecond(4, 1)
+				.projectFirst(0);
+
+		List<Tuple6<String, String, Long, Long, Long, Integer>> result = crossDs.collect();
+
+		String expected = "Hallo,Hi,1,1,1,1\n" +
+				"Hallo Welt,Hi,1,2,2,1\n" +
+				"Hallo Welt wie,Hi,1,1,3,1\n" +
+				"Hallo,Hello,2,1,1,2\n" +
+				"Hallo Welt,Hello,2,2,2,2\n" +
+				"Hallo Welt wie,Hello,2,1,3,2\n" +
+				"Hallo,Hello world,2,1,1,3\n" +
+				"Hallo Welt,Hello world,2,2,2,3\n" +
+				"Hallo Welt wie,Hello world,2,1,3,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDefaultCross() throws Exception {
+		/*
+		 * check correctness of default cross
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> crossDs = ds.cross(ds2);
+
+		List<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> result = crossDs.collect();
+
+		String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n"
+				+
+				"(1,1,Hi),(1,1,0,Hallo,1)\n" +
+				"(1,1,Hi),(2,3,2,Hallo Welt wie,1)\n" +
+				"(2,2,Hello),(2,2,1,Hallo Welt,2)\n" +
+				"(2,2,Hello),(1,1,0,Hallo,1)\n" +
+				"(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" +
+				"(3,2,Hello world),(2,2,1,Hallo Welt,2)\n" +
+				"(3,2,Hello world),(1,1,0,Hallo,1)\n" +
+				"(3,2,Hello world),(2,3,2,Hallo Welt wie,1)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossOnTwoCustomTypeInputs() throws Exception {
+		/*
+		 * check correctness of cross on two custom type inputs
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<CustomType> crossDs = ds.cross(ds2).with(new CustomTypeCross());
+
+		List<CustomType> result = crossDs.collect();
+
+		String expected = "1,0,HiHi\n"
+				+ "2,1,HiHello\n"
+				+ "2,2,HiHello world\n"
+				+ "2,1,HelloHi\n"
+				+ "4,2,HelloHello\n"
+				+ "4,3,HelloHello world\n"
+				+ "2,2,Hello worldHi\n"
+				+ "4,3,Hello worldHello\n"
+				+ "4,4,Hello worldHello world";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfCrossATupleInputAndACustomTypeInput() throws Exception {
+		/*
+		 * check correctness of cross a tuple input and a custom type input
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> crossDs = ds.cross(ds2).with(new MixedCross());
+
+		List<Tuple3<Integer, Long, String>> result = crossDs.collect();
+
+		String expected = "2,0,HalloHi\n" +
+				"3,0,HalloHello\n" +
+				"3,0,HalloHello world\n" +
+				"3,0,Hallo WeltHi\n" +
+				"4,1,Hallo WeltHello\n" +
+				"4,2,Hallo WeltHello world\n" +
+				"3,0,Hallo Welt wieHi\n" +
+				"4,2,Hallo Welt wieHello\n" +
+				"4,4,Hallo Welt wieHello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Tuple5Cross implements CrossFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, String> cross(
+				Tuple5<Integer, Long, Integer, String, Long> first,
+				Tuple5<Integer, Long, Integer, String, Long> second)
+				throws Exception {
+
+				return new Tuple2<Integer, String>(first.f2 + second.f2, first.f3 + second.f3);
+		}
+
+	}
+
+	private static class CustomTypeCross implements CrossFunction<CustomType, CustomType, CustomType> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public CustomType cross(CustomType first, CustomType second)
+				throws Exception {
+
+			return new CustomType(first.myInt * second.myInt, first.myLong + second.myLong, first.myString + second.myString);
+		}
+
+	}
+
+	private static class MixedCross implements CrossFunction<Tuple5<Integer, Long, Integer, String, Long>, CustomType, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple3<Integer, Long, String> cross(
+				Tuple5<Integer, Long, Integer, String, Long> first,
+				CustomType second) throws Exception {
+
+			return new Tuple3<Integer, Long, String>(first.f0 + second.myInt, first.f2 * second.myLong, first.f3 + second.myString);
+		}
+
+	}
+
+	private static class Tuple3ReturnLeft implements CrossFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple3<Integer, Long, String> cross(
+				Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second) throws Exception {
+
+			return first;
+		}
+	}
+
+	private static class Tuple5ReturnRight implements CrossFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple5<Integer, Long, Integer, String, Long> cross(
+				Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second)
+				throws Exception {
+
+			return second;
+		}
+
+	}
+
+	private static class Tuple5CrossBC extends RichCrossFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Integer, Integer>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private int broadcast = 42;
+
+		@Override
+		public void open(Configuration config) {
+
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			broadcast = sum;
+
+		}
+
+		@Override
+		public Tuple3<Integer, Integer, Integer> cross(
+				Tuple5<Integer, Long, Integer, String, Long> first,
+				Tuple5<Integer, Long, Integer, String, Long> second)
+				throws Exception {
+
+			return new Tuple3<Integer, Integer, Integer>(first.f0 + second.f0, first.f2 * second.f2, broadcast);
+		}
+	}
+}


[22/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
[FLINK-6731] [tests] Activate strict checkstyle for flink-tests

This closes #4295


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

Branch: refs/heads/master
Commit: 9bd491e05120915cbde36d4452e3982fe5d0975f
Parents: 480ccfb
Author: Greg Hogan <co...@greghogan.com>
Authored: Tue May 30 15:40:47 2017 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Wed Jul 12 18:37:47 2017 -0400

----------------------------------------------------------------------
 .../api/java/batch/TableEnvironmentITCase.java  |    2 +-
 .../api/java/batch/sql/GroupingSetsITCase.java  |    2 +-
 .../table/api/java/batch/sql/SqlITCase.java     |    2 +-
 flink-tests/pom.xml                             |   35 +
 .../accumulators/AccumulatorErrorITCase.java    |    8 +-
 .../test/accumulators/AccumulatorITCase.java    |   72 +-
 .../AccumulatorIterativeITCase.java             |   21 +-
 .../accumulators/AccumulatorLiveITCase.java     |   54 +-
 .../flink/test/actions/CountCollectITCase.java  |   14 +-
 .../broadcastvars/BroadcastBranchingITCase.java |   35 +-
 .../broadcastvars/BroadcastUnionITCase.java     |   14 +-
 .../BroadcastVarInitializationITCase.java       |   43 +-
 .../test/cancelling/CancelingTestBase.java      |   21 +-
 .../test/cancelling/JoinCancelingITCase.java    |   42 +-
 .../test/cancelling/MapCancelingITCase.java     |   28 +-
 ...tractEventTimeWindowCheckpointingITCase.java |   87 +-
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 .../CoStreamCheckpointingITCase.java            |   31 +-
 ...ontinuousFileProcessingCheckpointITCase.java |   18 +-
 .../EventTimeAllWindowCheckpointingITCase.java  |  115 +-
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 .../KeyedStateCheckpointingITCase.java          |   17 +-
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 .../test/checkpointing/RescalingITCase.java     |   46 +-
 ...ckendEventTimeWindowCheckpointingITCase.java |    3 +
 .../test/checkpointing/SavepointITCase.java     |   54 +-
 .../checkpointing/StateCheckpointedITCase.java  |   71 +-
 .../StreamCheckpointNotifierITCase.java         |   98 +-
 .../StreamCheckpointingITCase.java              |   46 +-
 .../StreamFaultToleranceTestBase.java           |   11 +-
 .../TimestampedFileInputSplitTest.java          |    5 +
 .../UdfStreamOperatorCheckpointingITCase.java   |   15 +-
 .../WindowCheckpointingITCase.java              |   49 +-
 .../utils/SavepointMigrationTestBase.java       |   22 +-
 ...atefulJobSavepointFrom11MigrationITCase.java |   50 +-
 ...atefulJobSavepointFrom12MigrationITCase.java |   52 +-
 .../test/classloading/ClassLoaderITCase.java    |   17 +-
 .../jar/CheckpointedStreamingProgram.java       |   24 +-
 .../jar/CheckpointingCustomKvStateProgram.java  |    8 +-
 .../jar/CustomInputSplitProgram.java            |   21 +-
 .../classloading/jar/CustomKvStateProgram.java  |    3 +-
 .../test/classloading/jar/KMeansForTest.java    |   41 +-
 .../jar/LegacyCheckpointedStreamingProgram.java |   19 +-
 .../jar/StreamingCustomInputSplitProgram.java   |   15 +-
 .../test/classloading/jar/StreamingProgram.java |   18 +-
 .../test/classloading/jar/UserCodeType.java     |    6 +-
 .../clients/examples/JobRetrievalITCase.java    |  140 --
 .../clients/examples/LocalExecutorITCase.java   |   76 -
 .../completeness/TypeInfoTestCoverageTest.java  |    9 +-
 .../distributedCache/DistributedCacheTest.java  |  123 --
 .../distributedcache/DistributedCacheTest.java  |  125 ++
 .../test/example/client/JobRetrievalITCase.java |  143 ++
 .../example/client/LocalExecutorITCase.java     |   81 +
 .../failing/JobSubmissionFailsITCase.java       |  208 +++
 .../test/example/failing/TaskFailureITCase.java |   98 ++
 .../example/java/ConnectedComponentsITCase.java |   65 +
 .../example/java/EnumTriangleBasicITCase.java   |   52 +
 .../flink/test/example/java/PageRankITCase.java |   95 +
 .../example/java/TransitiveClosureITCase.java   |   64 +
 .../test/example/java/WebLogAnalysisITCase.java |   57 +
 .../test/example/java/WordCountITCase.java      |   51 +
 .../example/java/WordCountNestedPOJOITCase.java |  129 ++
 .../example/java/WordCountSimplePOJOITCase.java |  110 ++
 .../WordCountSubclassInterfacePOJOITCase.java   |  169 ++
 .../java/WordCountSubclassPOJOITCase.java       |  133 ++
 .../java/WordCountWithCollectionITCase.java     |   70 +
 .../scala/ConnectedComponentsITCase.java        |   65 +
 .../test/example/scala/EnumTriangleITCase.java  |   52 +
 .../test/example/scala/PageRankITCase.java      |  100 ++
 .../example/scala/TransitiveClosureITCase.java  |   64 +
 .../example/scala/WebLogAnalysisITCase.java     |   57 +
 .../test/example/scala/WordCountITCase.java     |   57 +
 .../ConnectedComponentsITCase.java              |   63 -
 .../EnumTriangleBasicITCase.java                |   48 -
 .../exampleJavaPrograms/PageRankITCase.java     |   92 -
 .../TransitiveClosureITCase.java                |   63 -
 .../WebLogAnalysisITCase.java                   |   53 -
 .../exampleJavaPrograms/WordCountITCase.java    |   47 -
 .../WordCountNestedPOJOITCase.java              |  118 --
 .../WordCountSimplePOJOITCase.java              |  102 --
 .../WordCountSubclassInterfacePOJOITCase.java   |  152 --
 .../WordCountSubclassPOJOITCase.java            |  123 --
 .../WordCountWithCollectionITCase.java          |   66 -
 .../ConnectedComponentsITCase.java              |   64 -
 .../EnumTriangleITCase.java                     |   48 -
 .../exampleScalaPrograms/PageRankITCase.java    |   95 -
 .../TransitiveClosureITCase.java                |   63 -
 .../WebLogAnalysisITCase.java                   |   53 -
 .../exampleScalaPrograms/WordCountITCase.java   |   53 -
 .../JobSubmissionFailsITCase.java               |  204 ---
 .../test/failingPrograms/TaskFailureITCase.java |   98 --
 .../hadoop/mapred/HadoopIOFormatsITCase.java    |   55 +-
 .../hadoop/mapred/WordCountMapredITCase.java    |   11 +-
 .../mapreduce/WordCountMapreduceITCase.java     |   11 +-
 .../apache/flink/test/io/CsvReaderITCase.java   |   11 +-
 .../apache/flink/test/io/InputOutputITCase.java |    2 +-
 .../flink/test/io/RichInputOutputITCase.java    |    8 +-
 .../BulkIterationWithAllReducerITCase.java      |   43 +-
 .../CoGroupConnectedComponentsITCase.java       |   25 +-
 .../CoGroupConnectedComponentsSecondITCase.java |   73 +-
 .../iterative/ConnectedComponentsITCase.java    |   19 +-
 ...ectedComponentsWithDeferredUpdateITCase.java |   34 +-
 .../ConnectedComponentsWithObjectMapITCase.java |   34 +-
 ...tedComponentsWithSolutionSetFirstITCase.java |   19 +-
 .../test/iterative/DanglingPageRankITCase.java  |   83 +-
 ...terationNotDependingOnSolutionSetITCase.java |   22 +-
 .../DependencyConnectedComponentsITCase.java    |  184 +-
 .../iterative/EmptyWorksetIterationITCase.java  |   25 +-
 .../test/iterative/IdentityIterationITCase.java |   23 +-
 ...nIncompleteDynamicPathConsumptionITCase.java |   39 +-
 ...onIncompleteStaticPathConsumptionITCase.java |   39 +-
 ...IterationTerminationWithTerminationTail.java |   10 +-
 .../IterationTerminationWithTwoTails.java       |   11 +-
 .../IterationWithAllReducerITCase.java          |    6 +-
 .../iterative/IterationWithChainingITCase.java  |   15 +-
 .../iterative/IterationWithUnionITCase.java     |   24 +-
 .../iterative/KMeansWithBroadcastSetITCase.java |    7 +-
 .../MultipleSolutionSetJoinsITCase.java         |   37 +-
 .../iterative/SolutionSetDuplicatesITCase.java  |    6 +-
 .../StaticlyNestedIterationsITCase.java         |   35 +-
 .../UnionStaticDynamicIterationITCase.java      |   27 +-
 .../AggregatorConvergenceITCase.java            |  294 ++--
 .../aggregators/AggregatorsITCase.java          |   67 +-
 .../test/javaApiOperators/AggregateITCase.java  |  180 --
 .../CoGroupGroupSortITCase.java                 |  122 --
 .../test/javaApiOperators/CoGroupITCase.java    |  992 -----------
 .../test/javaApiOperators/CrossITCase.java      |  456 -----
 .../CustomDistributionITCase.java               |  359 ----
 .../test/javaApiOperators/DataSinkITCase.java   |  355 ----
 .../test/javaApiOperators/DataSourceITCase.java |   81 -
 .../test/javaApiOperators/DistinctITCase.java   |  318 ----
 .../ExecutionEnvironmentITCase.java             |   95 -
 .../test/javaApiOperators/FilterITCase.java     |  327 ----
 .../test/javaApiOperators/FirstNITCase.java     |  151 --
 .../test/javaApiOperators/FlatMapITCase.java    |  360 ----
 .../javaApiOperators/GroupCombineITCase.java    |  482 ------
 .../javaApiOperators/GroupReduceITCase.java     | 1636 ------------------
 .../flink/test/javaApiOperators/JoinITCase.java |  938 ----------
 .../flink/test/javaApiOperators/MapITCase.java  |  514 ------
 .../javaApiOperators/MapPartitionITCase.java    |  101 --
 .../javaApiOperators/ObjectReuseITCase.java     |  216 ---
 .../test/javaApiOperators/OuterJoinITCase.java  |  680 --------
 .../test/javaApiOperators/PartitionITCase.java  |  848 ---------
 .../test/javaApiOperators/ProjectITCase.java    |   64 -
 .../test/javaApiOperators/ReduceITCase.java     |  512 ------
 .../ReduceWithCombinerITCase.java               |  313 ----
 .../RemoteEnvironmentITCase.java                |  153 --
 .../ReplicatingDataSourceITCase.java            |  121 --
 .../test/javaApiOperators/SampleITCase.java     |  167 --
 .../javaApiOperators/SortPartitionITCase.java   |  343 ----
 .../test/javaApiOperators/SumMinMaxITCase.java  |  103 --
 .../test/javaApiOperators/TypeHintITCase.java   |  326 ----
 .../test/javaApiOperators/UnionITCase.java      |  132 --
 .../util/CollectionDataSets.java                |  725 --------
 .../util/ValueCollectionDataSets.java           |  730 --------
 .../test/manual/CheckForbiddenMethodsUsage.java |    7 +-
 .../HashTableRecordWidthCombinations.java       |   19 +-
 .../flink/test/manual/MassiveStringSorting.java |  161 +-
 .../test/manual/MassiveStringValueSorting.java  |  166 +-
 .../test/manual/NotSoMiniClusterIterations.java |    8 +-
 .../flink/test/manual/OverwriteObjects.java     |    7 +-
 .../flink/test/manual/ReducePerformance.java    |    7 +-
 .../manual/StreamingScalabilityAndLatency.java  |   48 +-
 .../apache/flink/test/manual/package-info.java  |    3 +-
 .../flink/test/misc/AutoParallelismITCase.java  |    8 +-
 .../test/misc/CustomPartitioningITCase.java     |   22 +-
 .../test/misc/CustomSerializationITCase.java    |   30 +-
 .../flink/test/misc/GenericTypeInfoTest.java    |   13 +-
 .../test/misc/MiscellaneousIssuesITCase.java    |   32 +-
 ...SuccessAfterNetworkBuffersFailureITCase.java |   26 +-
 .../flink/test/operators/AggregateITCase.java   |  183 ++
 .../test/operators/CoGroupGroupSortITCase.java  |  125 ++
 .../flink/test/operators/CoGroupITCase.java     |  989 +++++++++++
 .../flink/test/operators/CrossITCase.java       |  457 +++++
 .../operators/CustomDistributionITCase.java     |  362 ++++
 .../flink/test/operators/DataSinkITCase.java    |  356 ++++
 .../flink/test/operators/DataSourceITCase.java  |   82 +
 .../flink/test/operators/DistinctITCase.java    |  322 ++++
 .../operators/ExecutionEnvironmentITCase.java   |   95 +
 .../flink/test/operators/FilterITCase.java      |  331 ++++
 .../flink/test/operators/FirstNITCase.java      |  156 ++
 .../flink/test/operators/FlatMapITCase.java     |  364 ++++
 .../test/operators/GroupCombineITCase.java      |  484 ++++++
 .../flink/test/operators/GroupReduceITCase.java | 1633 +++++++++++++++++
 .../apache/flink/test/operators/JoinITCase.java |  945 ++++++++++
 .../apache/flink/test/operators/MapITCase.java  |  518 ++++++
 .../test/operators/MapPartitionITCase.java      |  101 ++
 .../flink/test/operators/ObjectReuseITCase.java |  215 +++
 .../flink/test/operators/OuterJoinITCase.java   |  682 ++++++++
 .../flink/test/operators/PartitionITCase.java   |  847 +++++++++
 .../flink/test/operators/ProjectITCase.java     |   67 +
 .../flink/test/operators/ReduceITCase.java      |  515 ++++++
 .../operators/ReduceWithCombinerITCase.java     |  317 ++++
 .../test/operators/RemoteEnvironmentITCase.java |  157 ++
 .../operators/ReplicatingDataSourceITCase.java  |  118 ++
 .../flink/test/operators/SampleITCase.java      |  171 ++
 .../test/operators/SortPartitionITCase.java     |  347 ++++
 .../flink/test/operators/SumMinMaxITCase.java   |  108 ++
 .../flink/test/operators/TypeHintITCase.java    |  330 ++++
 .../flink/test/operators/UnionITCase.java       |  136 ++
 .../test/operators/util/CollectionDataSets.java |  772 +++++++++
 .../operators/util/ValueCollectionDataSets.java |  775 +++++++++
 .../examples/KMeansSingleStepTest.java          |   80 +-
 .../examples/RelationalQueryCompilerTest.java   |  156 +-
 .../examples/WordCountCompilerTest.java         |   20 +-
 .../ConnectedComponentsCoGroupTest.java         |   64 +-
 ...ultipleJoinsWithSolutionSetCompilerTest.java |   82 +-
 .../iterations/PageRankCompilerTest.java        |   63 +-
 .../jsonplan/DumpCompiledPlanTest.java          |   39 +-
 .../jsonplan/JsonJobGraphGenerationTest.java    |   87 +-
 .../optimizer/jsonplan/PreviewPlanDumpTest.java |   39 +-
 .../query/AbstractQueryableStateITCase.java     |   52 +-
 .../KVStateRequestSerializerRocksDBTest.java    |   15 +-
 .../query/QueryableStateITCaseFsBackend.java    |    1 +
 .../QueryableStateITCaseRocksDBBackend.java     |    1 +
 ...ctTaskManagerProcessFailureRecoveryTest.java |   33 +-
 .../flink/test/recovery/FastFailuresITCase.java |   15 +-
 .../JobManagerHACheckpointRecoveryITCase.java   |   72 +-
 .../JobManagerHAJobGraphRecoveryITCase.java     |   34 +-
 ...agerHAProcessFailureBatchRecoveryITCase.java |   24 +-
 .../recovery/ProcessFailureCancelingITCase.java |   60 +-
 ...SimpleRecoveryFailureRateStrategyITBase.java |    4 +
 ...RecoveryFixedDelayRestartStrategyITBase.java |    4 +
 .../test/recovery/SimpleRecoveryITCaseBase.java |   22 +-
 .../TaskManagerFailureRecoveryITCase.java       |   38 +-
 ...anagerProcessFailureBatchRecoveryITCase.java |    7 +-
 ...erProcessFailureStreamingRecoveryITCase.java |   24 +-
 ...ConsumePipelinedAndBlockingResultITCase.java |    5 +-
 .../flink/test/runtime/IPv6HostnamesITCase.java |   36 +-
 .../flink/test/runtime/JoinDeadlockITCase.java  |    3 +-
 .../runtime/NetworkStackThroughputITCase.java   |   15 +-
 .../RegisterTypeWithKryoSerializerITCase.java   |   16 +-
 .../test/runtime/SelfJoinDeadlockITCase.java    |    3 +-
 .../ZooKeeperLeaderElectionITCase.java          |   19 +-
 .../LocalFlinkMiniClusterITCase.java            |   43 +-
 .../test/state/ManualWindowSpeedITCase.java     |    4 +-
 .../state/StateHandleSerializationTest.java     |   20 +-
 .../AbstractOperatorRestoreTestBase.java        |   21 +-
 .../state/operator/restore/ExecutionMode.java   |    3 +-
 .../restore/keyed/KeyedComplexChainTest.java    |    4 +
 .../state/operator/restore/keyed/KeyedJob.java  |    4 +-
 ...AbstractNonKeyedOperatorRestoreTestBase.java |    1 +
 .../restore/unkeyed/ChainBreakTest.java         |    1 +
 .../unkeyed/ChainLengthDecreaseTest.java        |    1 +
 .../unkeyed/ChainLengthIncreaseTest.java        |    1 +
 .../restore/unkeyed/ChainOrderTest.java         |    1 +
 .../restore/unkeyed/ChainUnionTest.java         |    1 +
 .../operator/restore/unkeyed/NonKeyedJob.java   |    4 +-
 .../streaming/api/StreamingOperatorsITCase.java |   31 +-
 .../api/outputformat/CsvOutputFormatITCase.java |    3 +
 .../outputformat/TextOutputFormatITCase.java    |    3 +
 .../runtime/ChainedRuntimeContextITCase.java    |    3 +
 .../streaming/runtime/CoGroupJoinITCase.java    |   18 +-
 .../test/streaming/runtime/CoStreamITCase.java  |    8 +-
 .../streaming/runtime/DataStreamPojoITCase.java |   65 +-
 .../streaming/runtime/DirectedOutputITCase.java |    3 +
 .../test/streaming/runtime/IterateITCase.java   |  115 +-
 .../streaming/runtime/OutputSplitterITCase.java |    3 +
 .../streaming/runtime/PartitionerITCase.java    |    2 -
 .../streaming/runtime/SelfConnectionITCase.java |    4 +-
 .../streaming/runtime/SideOutputITCase.java     |   25 +-
 .../streaming/runtime/StateBackendITCase.java   |    9 +-
 .../runtime/StreamTaskTimerITCase.java          |   22 +-
 .../test/streaming/runtime/TimestampITCase.java |  156 +-
 .../streaming/runtime/WindowFoldITCase.java     |    3 +-
 .../runtime/util/EvenOddOutputSelector.java     |    4 +
 .../test/streaming/runtime/util/NoOpIntMap.java |    4 +
 .../runtime/util/ReceiveCheckNoOpSink.java      |    6 +
 .../runtime/util/TestListResultSink.java        |    9 +-
 .../streaming/runtime/util/TestListWrapper.java |    5 +-
 .../flink/test/testfunctions/Tokenizer.java     |    3 +
 .../PojoSerializerUpgradeTest.java              |   23 +-
 .../org/apache/flink/test/util/CoordVector.java |   24 +-
 .../flink/test/util/DataSetUtilsITCase.java     |   36 +-
 .../test/util/InfiniteIntegerInputFormat.java   |    7 +-
 .../util/InfiniteIntegerTupleInputFormat.java   |    7 +-
 .../apache/flink/test/util/PointFormatter.java  |    6 +-
 .../apache/flink/test/util/PointInFormat.java   |    8 +-
 .../org/apache/flink/test/util/TestUtils.java   |    7 +-
 .../UniformIntTupleGeneratorInputFormat.java    |    8 +-
 .../sessionwindows/EventGenerator.java          |    2 +-
 .../sessionwindows/EventGeneratorFactory.java   |    3 +-
 .../sessionwindows/GeneratorConfiguration.java  |   22 +-
 .../sessionwindows/GeneratorEventFactory.java   |   15 +-
 .../sessionwindows/LongRandomGenerator.java     |    7 +-
 .../ParallelSessionsEventGenerator.java         |   14 +-
 .../sessionwindows/SessionConfiguration.java    |    4 +-
 .../windowing/sessionwindows/SessionEvent.java  |    2 +-
 .../SessionEventGeneratorImpl.java              |   12 +-
 .../SessionGeneratorConfiguration.java          |    2 +-
 .../sessionwindows/SessionWindowITCase.java     |   12 +-
 .../sessionwindows/TestEventPayload.java        |    4 +-
 .../scala/operators/GroupCombineITCase.scala    |    2 +-
 295 files changed, 17887 insertions(+), 17270 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
index aac7e11..0def39a 100644
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
@@ -35,7 +35,7 @@ import org.apache.flink.table.api.scala.batch.utils.TableProgramsCollectionTestB
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
 import org.apache.flink.table.calcite.CalciteConfig;
 import org.apache.flink.table.calcite.CalciteConfigBuilder;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
 import org.apache.flink.types.Row;
 
 import org.apache.calcite.tools.RuleSets;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsITCase.java
index 6c1a753..29fbdf5 100644
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsITCase.java
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/GroupingSetsITCase.java
@@ -28,7 +28,7 @@ import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableEnvironment;
 import org.apache.flink.table.api.java.BatchTableEnvironment;
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsClusterTestBase;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
index f4e5daf..6a72b3e 100644
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
 import org.apache.flink.table.api.java.BatchTableEnvironment;
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsCollectionTestBase;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
 import org.apache.flink.types.Row;
 
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index beac803..3f67a88 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -338,6 +338,41 @@ under the License.
 				</executions>
 			</plugin>
 
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-checkstyle-plugin</artifactId>
+				<version>2.17</version>
+				<dependencies>
+					<dependency>
+						<groupId>com.puppycrawl.tools</groupId>
+						<artifactId>checkstyle</artifactId>
+						<version>6.19</version>
+					</dependency>
+				</dependencies>
+				<configuration>
+					<configLocation>/tools/maven/strict-checkstyle.xml</configLocation>
+					<suppressionsLocation>/tools/maven/suppressions.xml</suppressionsLocation>
+					<includeTestSourceDirectory>true</includeTestSourceDirectory>
+					<logViolationsToConsole>true</logViolationsToConsole>
+					<failOnViolation>true</failOnViolation>
+				</configuration>
+				<executions>
+					<!--
+						Execute checkstyle after compilation but before tests.
+
+						This ensures that any parsing or type checking errors are from
+						javac, so they look as expected. Beyond that, we want to
+						fail as early as possible.
+					-->
+					<execution>
+						<phase>test-compile</phase>
+						<goals>
+							<goal>check</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
 			<!-- Scala Code Style, most of the configuration done via plugin management -->
 			<plugin>
 				<groupId>org.scalastyle</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
index 25d9228..4de1602 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
@@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
@@ -32,18 +31,18 @@ import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-
 import static org.junit.Assert.fail;
 
 /**
- * Tests cases where Accumulator are
+ * Tests cases where accumulators:
  *  a) throw errors during runtime
- *  b) is not compatible with existing accumulator
+ *  b) are not compatible with existing accumulator.
  */
 public class AccumulatorErrorITCase extends TestLogger {
 
@@ -91,7 +90,6 @@ public class AccumulatorErrorITCase extends TestLogger {
 		}
 	}
 
-
 	@Test
 	public void testInvalidTypeAccumulator() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
index 5f2b0a9..3e35bd0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.test.accumulators;
 
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
@@ -38,16 +34,20 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.Collector;
-import org.junit.Assert;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import org.junit.Assert;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * Test for the basic functionality of accumulators. We cannot test all different
  * kinds of plans here (iterative, etc.).
- * 
- * TODO Test conflict when different UDFs write to accumulator with same name
+ *
+ * <p>TODO Test conflict when different UDFs write to accumulator with same name
  * but with different type. The conflict will occur in JobManager while merging.
  */
 @SuppressWarnings("serial")
@@ -60,31 +60,31 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 	private String resultPath;
 
 	private JobExecutionResult result;
-	
+
 	@Override
 	protected void preSubmit() throws Exception {
 		dataPath = createTempFile("datapoints.txt", INPUT);
 		resultPath = getTempFilePath("result");
 	}
-	
+
 	@Override
 	protected void postSubmit() throws Exception {
 		compareResultsByLinesInMemory(EXPECTED, resultPath);
-		
+
 		// Test accumulator results
 		System.out.println("Accumulator results:");
 		JobExecutionResult res = this.result;
 		System.out.println(AccumulatorHelper.getResultsFormatted(res.getAllAccumulatorResults()));
 
-		Assert.assertEquals(Integer.valueOf(3), (Integer) res.getAccumulatorResult("num-lines"));
+		Assert.assertEquals(Integer.valueOf(3), res.getAccumulatorResult("num-lines"));
+
+		Assert.assertEquals(Double.valueOf(getParallelism()), res.getAccumulatorResult("open-close-counter"));
 
-		Assert.assertEquals(Double.valueOf(getParallelism()), (Double)res.getAccumulatorResult("open-close-counter"));
-		
 		// Test histogram (words per line distribution)
 		Map<Integer, Integer> dist = Maps.newHashMap();
 		dist.put(1, 1); dist.put(2, 1); dist.put(3, 1);
 		Assert.assertEquals(dist, res.getAccumulatorResult("words-per-line"));
-		
+
 		// Test distinct words (custom accumulator)
 		Set<StringValue> distinctWords = Sets.newHashSet();
 		distinctWords.add(new StringValue("one"));
@@ -96,18 +96,18 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		DataSet<String> input = env.readTextFile(dataPath); 
-		
+
+		DataSet<String> input = env.readTextFile(dataPath);
+
 		input.flatMap(new TokenizeLine())
 			.groupBy(0)
 			.reduceGroup(new CountWords())
 			.writeAsCsv(resultPath, "\n", " ");
-		
+
 		this.result = env.execute();
 	}
-	
-	public static class TokenizeLine extends RichFlatMapFunction<String, Tuple2<String, Integer>> {
+
+	private static class TokenizeLine extends RichFlatMapFunction<String, Tuple2<String, Integer>> {
 
 		// Needs to be instantiated later since the runtime context is not yet
 		// initialized at this place
@@ -120,7 +120,7 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 
 		@Override
 		public void open(Configuration parameters) {
-		  
+
 			// Add counters using convenience functions
 			this.cntNumLines = getRuntimeContext().getIntCounter("num-lines");
 			this.wordsPerLineDistribution = getRuntimeContext().getHistogram("words-per-line");
@@ -157,20 +157,20 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 			// Test counter used in open() and closed()
 			this.openCloseCounter.add(0.5);
 		}
-		
+
 		@Override
 		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
 			this.cntNumLines.add(1);
 			int wordsPerLine = 0;
-			
+
 			for (String token : value.toLowerCase().split("\\W+")) {
 				distinctWords.add(new StringValue(token));
 				out.collect(new Tuple2<>(token, 1));
-				++ wordsPerLine;
+				++wordsPerLine;
 			}
 			wordsPerLineDistribution.add(wordsPerLine);
 		}
-		
+
 		@Override
 		public void close() throws Exception {
 			// Test counter used in open and close only
@@ -179,37 +179,35 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 		}
 	}
 
-	
-	public static class CountWords
+	private static class CountWords
 		extends RichGroupReduceFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>
-		implements GroupCombineFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>
-	{
-		
+		implements GroupCombineFunction<Tuple2<String, Integer>, Tuple2<String, Integer>> {
+
 		private IntCounter reduceCalls;
 		private IntCounter combineCalls;
-		
+
 		@Override
 		public void open(Configuration parameters) {
 			this.reduceCalls = getRuntimeContext().getIntCounter("reduce-calls");
 			this.combineCalls = getRuntimeContext().getIntCounter("combine-calls");
 		}
-		
+
 		@Override
 		public void reduce(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
 			reduceCalls.add(1);
 			reduceInternal(values, out);
 		}
-		
+
 		@Override
 		public void combine(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
 			combineCalls.add(1);
 			reduceInternal(values, out);
 		}
-		
+
 		private void reduceInternal(Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) {
 			int sum = 0;
 			String key = null;
-			
+
 			for (Tuple2<String, Integer> e : values) {
 				key = e.f0;
 				sum += e.f1;
@@ -217,9 +215,9 @@ public class AccumulatorITCase extends JavaProgramTestBase {
 			out.collect(new Tuple2<>(key, sum));
 		}
 	}
-	
+
 	/**
-	 * Custom accumulator
+	 * Custom accumulator.
 	 */
 	public static class SetAccumulator<T> implements Accumulator<T, HashSet<T>> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
index d86d517..3e19ce8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.accumulators;
 
 import org.apache.flink.api.common.accumulators.IntCounter;
@@ -27,13 +26,17 @@ import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 
-public class AccumulatorIterativeITCase extends JavaProgramTestBase {	
+/**
+ * Test accumulator within iteration.
+ */
+public class AccumulatorIterativeITCase extends JavaProgramTestBase {
 	private static final int NUM_ITERATIONS = 3;
 	private static final int NUM_SUBTASKS = 1;
 	private static final String ACC_NAME = "test";
-	
+
 	@Override
 	protected boolean skipCollectionExecution() {
 		return true;
@@ -43,20 +46,20 @@ public class AccumulatorIterativeITCase extends JavaProgramTestBase {
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(NUM_SUBTASKS);
-		
+
 		IterativeDataSet<Integer> iteration = env.fromElements(1, 2, 3).iterate(NUM_ITERATIONS);
-		
+
 		iteration.closeWith(iteration.reduceGroup(new SumReducer())).output(new DiscardingOutputFormat<Integer>());
 
 		Assert.assertEquals(NUM_ITERATIONS * 6, (int) env.execute().getAccumulatorResult(ACC_NAME));
 	}
-	
+
 	static final class SumReducer extends RichGroupReduceFunction<Integer, Integer> {
-		
+
 		private static final long serialVersionUID = 1L;
-		
+
 		private IntCounter testCounter = new IntCounter();
-		
+
 		@Override
 		public void open(Configuration config) throws Exception {
 			getRuntimeContext().addAccumulator(ACC_NAME, this.testCounter);

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
index 92e5768..756b81e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.test.accumulators;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.pattern.Patterns;
-import akka.testkit.JavaTestKit;
-import akka.util.Timeout;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.Plan;
@@ -55,39 +49,42 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.util.Collector;
-
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import akka.testkit.JavaTestKit;
+import akka.util.Timeout;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.*;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
+import static org.junit.Assert.fail;
 
 /**
  * Tests the availability of accumulator results during runtime. The test case tests a user-defined
  * accumulator and Flink's internal accumulators for two consecutive tasks.
  *
- * CHAINED[Source -> Map] -> Sink
+ * <p>CHAINED[Source -> Map] -> Sink
  *
- * Checks are performed as the elements arrive at the operators. Checks consist of a message sent by
+ * <p>Checks are performed as the elements arrive at the operators. Checks consist of a message sent by
  * the task to the task manager which notifies the job manager and sends the current accumulators.
  * The task blocks until the test has been notified about the current accumulator values.
  *
- * A barrier between the operators ensures that that pipelining is disabled for the streaming test.
+ * <p>A barrier between the operators ensures that that pipelining is disabled for the streaming test.
  * The batch job reads the records one at a time. The streaming code buffers the records beforehand;
  * that's why exact guarantees about the number of records read are very hard to make. Thus, why we
  * check for an upper bound of the elements read.
@@ -104,7 +101,7 @@ public class AccumulatorLiveITCase extends TestLogger {
 	private static JobGraph jobGraph;
 
 	// name of user accumulator
-	private static String ACCUMULATOR_NAME = "test";
+	private static final String ACCUMULATOR_NAME = "test";
 
 	// number of heartbeat intervals to check
 	private static final int NUM_ITERATIONS = 5;
@@ -113,7 +110,6 @@ public class AccumulatorLiveITCase extends TestLogger {
 
 	private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS);
 
-
 	@Before
 	public void before() throws Exception {
 		system = AkkaUtils.createLocalActorSystem(new Configuration());
@@ -129,8 +125,8 @@ public class AccumulatorLiveITCase extends TestLogger {
 		taskManager = testingCluster.getTaskManagersAsJava().get(0);
 
 		// generate test data
-		for (int i=0; i < NUM_ITERATIONS; i++) {
-			inputData.add(i, String.valueOf(i+1));
+		for (int i = 0; i < NUM_ITERATIONS; i++) {
+			inputData.add(i, String.valueOf(i + 1));
 		}
 
 		NotifyingMapper.finished = false;
@@ -163,7 +159,6 @@ public class AccumulatorLiveITCase extends TestLogger {
 		verifyResults();
 	}
 
-
 	@Test
 	public void testStreaming() throws Exception {
 
@@ -175,14 +170,12 @@ public class AccumulatorLiveITCase extends TestLogger {
 				.flatMap(new NotifyingMapper())
 				.writeUsingOutputFormat(new NotifyingOutputFormat()).disableChaining();
 
-
 		jobGraph = env.getStreamGraph().getJobGraph();
 		jobID = jobGraph.getJobID();
 
 		verifyResults();
 	}
 
-
 	private static void verifyResults() {
 		new JavaTestKit(system) {{
 
@@ -201,7 +194,6 @@ public class AccumulatorLiveITCase extends TestLogger {
 					selfGateway);
 			expectMsgClass(TIMEOUT, JobManagerMessages.JobSubmitSuccess.class);
 
-
 			TestingJobManagerMessages.UpdatedAccumulators msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT);
 			Map<String, Accumulator<?, ?>> userAccumulators = msg.userAccumulators();
 
@@ -210,7 +202,7 @@ public class AccumulatorLiveITCase extends TestLogger {
 			ExecutionAttemptID sinkTaskID = null;
 
 			/* Check for accumulator values */
-			if(checkUserAccumulators(0, userAccumulators)) {
+			if (checkUserAccumulators(0, userAccumulators)) {
 				LOG.info("Passed initial check for map task.");
 			} else {
 				fail("Wrong accumulator results when map task begins execution.");
@@ -244,7 +236,7 @@ public class AccumulatorLiveITCase extends TestLogger {
 			msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT);
 			userAccumulators = msg.userAccumulators();
 
-			if(checkUserAccumulators(expectedAccVal, userAccumulators)) {
+			if (checkUserAccumulators(expectedAccVal, userAccumulators)) {
 				LOG.info("Passed initial check for sink task.");
 			} else {
 				fail("Wrong accumulator results when sink task begins execution.");
@@ -272,14 +264,13 @@ public class AccumulatorLiveITCase extends TestLogger {
 		}};
 	}
 
-
-	private static boolean checkUserAccumulators(int expected, Map<String, Accumulator<?,?>> accumulatorMap) {
+	private static boolean checkUserAccumulators(int expected, Map<String, Accumulator<?, ?>> accumulatorMap) {
 		LOG.info("checking user accumulators");
-		return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter)accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue();
+		return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter) accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue();
 	}
 
 	/**
-	 * UDF that notifies when it changes the accumulator values
+	 * UDF that notifies when it changes the accumulator values.
 	 */
 	private static class NotifyingMapper extends RichFlatMapFunction<String, Integer> {
 		private static final long serialVersionUID = 1L;
@@ -356,7 +347,7 @@ public class AccumulatorLiveITCase extends TestLogger {
 	}
 
 	/**
-	 * Helpers to generate the JobGraph
+	 * Helpers to generate the JobGraph.
 	 */
 	private static JobGraph getOptimizedPlan(Plan plan) {
 		Optimizer pc = new Optimizer(new DataStatistics(), new Configuration());
@@ -376,7 +367,6 @@ public class AccumulatorLiveITCase extends TestLogger {
 		}
 	}
 
-
 	/**
 	 * This is used to for creating the example topology. {@link #execute} is never called, we
 	 * only use this to call {@link #getStreamGraph()}.

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java b/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
index e742c27..737bdfb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/actions/CountCollectITCase.java
@@ -21,17 +21,18 @@ package org.apache.flink.test.actions;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple2;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import static org.junit.Assert.*;
-
 import org.apache.flink.test.util.MultipleProgramsTestBase;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
 /**
  * Tests the methods that bring elements back to the client driver program.
  */
@@ -64,7 +65,6 @@ public class CountCollectITCase extends MultipleProgramsTestBase {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableObjectReuse();
 
-
 		DataSet<Integer> data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
 		DataSet<Integer> data2 = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
index 5f2950c..720784e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.test.broadcastvars;
 
-import java.util.Collection;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -35,6 +32,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Test broadcast input after branching.
+ */
 public class BroadcastBranchingITCase extends JavaProgramTestBase {
 	private static final String RESULT = "(2,112)\n";
 
@@ -63,16 +66,16 @@ public class BroadcastBranchingITCase extends JavaProgramTestBase {
 				.fromElements(new Tuple2<>("1", 2), new Tuple2<>("2", 3), new Tuple2<>("3", 7));
 
 		// Jn1 matches x and y values on id and emits (id, x, y) triples
-		JoinOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> jn1
-				= sc2.join(sc3).where(0).equalTo(0).with(new Jn1());
+		JoinOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> jn1 =
+				sc2.join(sc3).where(0).equalTo(0).with(new Jn1());
 
 		// Jn2 matches polynomial and arguments by id, computes p = min(P(x),P(y)) and emits (id, p) tuples
-		JoinOperator<Tuple3<String, Integer, Integer>, Tuple4<String, Integer, Integer, Integer>, Tuple2<String, Integer>> jn2
-				= jn1.join(sc1).where(0).equalTo(0).with(new Jn2());
+		JoinOperator<Tuple3<String, Integer, Integer>, Tuple4<String, Integer, Integer, Integer>, Tuple2<String, Integer>> jn2 =
+				jn1.join(sc1).where(0).equalTo(0).with(new Jn2());
 
 		// Mp1 selects (id, x, y) triples where x = y and broadcasts z (=x=y) to Mp2
-		FlatMapOperator<Tuple3<String, Integer, Integer>, Tuple2<String, Integer>> mp1
-				= jn1.flatMap(new Mp1());
+		FlatMapOperator<Tuple3<String, Integer, Integer>, Tuple2<String, Integer>> mp1 =
+				jn1.flatMap(new Mp1());
 
 		// Mp2 filters out all p values which can be divided by z
 		List<Tuple2<String, Integer>> result = jn2.flatMap(new Mp2()).withBroadcastSet(mp1, "z").collect();
@@ -80,7 +83,7 @@ public class BroadcastBranchingITCase extends JavaProgramTestBase {
 		JavaProgramTestBase.compareResultAsText(result, RESULT);
 	}
 
-	public static class Jn1 implements JoinFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> {
+	private static class Jn1 implements JoinFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -89,7 +92,7 @@ public class BroadcastBranchingITCase extends JavaProgramTestBase {
 		}
 	}
 
-	public static class Jn2 implements JoinFunction<Tuple3<String, Integer, Integer>, Tuple4<String, Integer, Integer, Integer>, Tuple2<String, Integer>> {
+	private static class Jn2 implements JoinFunction<Tuple3<String, Integer, Integer>, Tuple4<String, Integer, Integer, Integer>, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		private static int p(int x, int a, int b, int c) {
@@ -104,14 +107,14 @@ public class BroadcastBranchingITCase extends JavaProgramTestBase {
 			int b = second.f2;
 			int c = second.f3;
 
-			int p_x = p(x, a, b, c);
-			int p_y = p(y, a, b, c);
-			int min = Math.min(p_x, p_y);
+			int pX = p(x, a, b, c);
+			int pY = p(y, a, b, c);
+			int min = Math.min(pX, pY);
 			return new Tuple2<>(first.f0, min);
 		}
 	}
 
-	public static class Mp1 implements FlatMapFunction<Tuple3<String, Integer, Integer>, Tuple2<String, Integer>> {
+	private static class Mp1 implements FlatMapFunction<Tuple3<String, Integer, Integer>, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -122,7 +125,7 @@ public class BroadcastBranchingITCase extends JavaProgramTestBase {
 		}
 	}
 
-	public static class Mp2 extends RichFlatMapFunction<Tuple2<String, Integer>, Tuple2<String, Integer>> {
+	private static class Mp2 extends RichFlatMapFunction<Tuple2<String, Integer>, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;
 
 		private Collection<Tuple2<String, Integer>> zs;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
index 113a330..79b0033 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java
@@ -15,17 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.broadcastvars;
 
-import java.util.List;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
+
 import org.junit.Assert;
 
+import java.util.List;
+
+/**
+ * Test broadcast input after union.
+ */
 public class BroadcastUnionITCase extends JavaProgramTestBase {
 	private static final String BC_NAME = "bc";
 
@@ -43,11 +49,11 @@ public class BroadcastUnionITCase extends JavaProgramTestBase {
 				.withBroadcastSet(bc1.union(bc2), BC_NAME)
 				.reduce(new Reducer())
 				.collect();
-		
+
 		Assert.assertEquals(Long.valueOf(3025), result.get(0));
 	}
 
-	public static class Mapper extends RichMapFunction<Long, Long> {
+	private static class Mapper extends RichMapFunction<Long, Long> {
 		private List<Long> values;
 
 		@Override
@@ -65,7 +71,7 @@ public class BroadcastUnionITCase extends JavaProgramTestBase {
 		}
 	}
 
-	public static class Reducer implements ReduceFunction<Long> {
+	private static class Reducer implements ReduceFunction<Long> {
 		@Override
 		public Long reduce(Long value1, Long value2) throws Exception {
 			return value1 + value2;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
index 41d24b8..26f5763 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.test.broadcastvars;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
 import org.apache.flink.api.java.DataSet;
@@ -31,36 +27,43 @@ import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Test {@link BroadcastVariableInitializer}.
+ */
 @SuppressWarnings("serial")
 public class BroadcastVarInitializationITCase extends JavaProgramTestBase {
-	
+
 	@Override
 	protected void testProgram() throws Exception {
-		
+
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(4);
-		
+
 		DataSet<Integer> data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8);
-		
+
 		IterativeDataSet<Integer> iteration = data.iterate(10);
-		
+
 		DataSet<Integer> result = data.reduceGroup(new PickOneAllReduce()).withBroadcastSet(iteration, "bc");
-		
+
 		final List<Integer> resultList = new ArrayList<Integer>();
 		iteration.closeWith(result).output(new LocalCollectionOutputFormat<Integer>(resultList));
-		
+
 		env.execute();
-		
+
 		Assert.assertEquals(8, resultList.get(0).intValue());
 	}
 
-	
-	public static class PickOneAllReduce extends RichGroupReduceFunction<Integer, Integer> {
-		
+	private static class PickOneAllReduce extends RichGroupReduceFunction<Integer, Integer> {
+
 		private Integer bcValue;
-		
+
 		@Override
 		public void open(Configuration parameters) {
 			this.bcValue = getRuntimeContext().getBroadcastVariableWithInitializer("bc", new PickFirstInitializer());
@@ -72,8 +75,8 @@ public class BroadcastVarInitializationITCase extends JavaProgramTestBase {
 				return;
 			}
 			final int x = bcValue;
-			
-			for (Integer y : records) { 
+
+			for (Integer y : records) {
 				if (y > x) {
 					out.collect(y);
 					return;
@@ -83,8 +86,8 @@ public class BroadcastVarInitializationITCase extends JavaProgramTestBase {
 			out.collect(bcValue);
 		}
 	}
-	
-	public static class PickFirstInitializer implements BroadcastVariableInitializer<Integer, Integer> {
+
+	private static class PickFirstInitializer implements BroadcastVariableInitializer<Integer, Integer> {
 
 		@Override
 		public Integer initializeBroadcastVariable(Iterable<Integer> data) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
index 2767312..88c921d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.cancelling;
 
 import org.apache.flink.api.common.Plan;
@@ -35,31 +34,33 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
 import org.apache.flink.util.TestLogger;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-import java.util.concurrent.TimeUnit;
-
 import static org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import static org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure;
 import static org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess;
 
 /**
- * 
+ * Base class for testing job cancellation.
  */
 public abstract class CancelingTestBase extends TestLogger {
-	
+
 	private static final Logger LOG = LoggerFactory.getLogger(CancelingTestBase.class);
 
 	private static final int MINIMUM_HEAP_SIZE_MB = 192;
-	
+
 	/**
 	 * Defines the number of seconds after which an issued cancel request is expected to have taken effect (i.e. the job
 	 * is canceled), starting from the point in time when the cancel request is issued.
@@ -69,13 +70,13 @@ public abstract class CancelingTestBase extends TestLogger {
 	private static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	protected LocalFlinkMiniCluster executor;
 
 	protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private void verifyJvmOptions() {
 		final long heap = Runtime.getRuntime().maxMemory() >> 20;
 		Assert.assertTrue("Insufficient java heap space " + heap + "mb - set JVM option: -Xmx" + MINIMUM_HEAP_SIZE_MB
@@ -112,7 +113,7 @@ public abstract class CancelingTestBase extends TestLogger {
 	public void runAndCancelJob(Plan plan, int msecsTillCanceling) throws Exception {
 		runAndCancelJob(plan, msecsTillCanceling, DEFAULT_CANCEL_FINISHED_INTERVAL);
 	}
-		
+
 	public void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxTimeTillCanceled) throws Exception {
 		try {
 			// submit job

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
index d797e47..5e21129 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/JoinCancelingITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.cancelling;
 
 import org.apache.flink.api.common.functions.JoinFunction;
@@ -31,13 +30,16 @@ import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
 import org.apache.flink.test.util.InfiniteIntegerTupleInputFormat;
 import org.apache.flink.test.util.UniformIntTupleGeneratorInputFormat;
 
+/**
+ * Test job cancellation from within a JoinFunction.
+ */
 public class JoinCancelingITCase extends CancelingTestBase {
 	private static final int parallelism = 4;
 
 	public JoinCancelingITCase() {
 		setTaskManagerNumSlots(parallelism);
 	}
-	
+
 	// --------------- Test Sort Matches that are canceled while still reading / sorting -----------------
 	private void executeTask(JoinFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> joiner, boolean slow) throws Exception {
 		executeTask(joiner, slow, parallelism);
@@ -68,7 +70,7 @@ public class JoinCancelingITCase extends CancelingTestBase {
 	public void testCancelSortMatchWhileReadingFastInputs() throws Exception {
 		executeTask(new SimpleMatcher<Integer>(), false);
 	}
-	
+
 //	@Test
 	public void testCancelSortMatchPriorToFirstRecordReading() throws Exception {
 		executeTask(new StuckInOpenMatcher<Integer>(), false);
@@ -92,45 +94,45 @@ public class JoinCancelingITCase extends CancelingTestBase {
 
 		runAndCancelJob(env.createProgramPlan(), msecsTillCanceling, maxTimeTillCanceled);
 	}
-	
+
 //	@Test
 	public void testCancelSortMatchWhileDoingHeavySorting() throws Exception {
 		executeTaskWithGenerator(new SimpleMatcher<Integer>(), 50000, 100, 30 * 1000, 30 * 1000);
 	}
 
 	// --------------- Test Sort Matches that are canceled while in the Matching Phase -----------------
-	
+
 //	@Test
 	public void testCancelSortMatchWhileJoining() throws Exception {
 		executeTaskWithGenerator(new DelayingMatcher<Integer>(), 500, 3, 10 * 1000, 20 * 1000);
 	}
-	
+
 //	@Test
 	public void testCancelSortMatchWithLongCancellingResponse() throws Exception {
 		executeTaskWithGenerator(new LongCancelTimeMatcher<Integer>(), 500, 3, 10 * 1000, 10 * 1000);
 	}
 
 	// -------------------------------------- Test System corner cases ---------------------------------
-	
+
 //	@Test
 	public void testCancelSortMatchWithHighparallelism() throws Exception {
 		executeTask(new SimpleMatcher<Integer>(), false, 64);
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
-	public static final class SimpleMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
+
+	private static final class SimpleMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Tuple2<IN, IN> join(Tuple2<IN, IN> first, Tuple2<IN, IN> second) throws Exception {
 			return new Tuple2<>(first.f0, second.f0);
 		}
 	}
-	
-	public static final class DelayingMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
+
+	private static final class DelayingMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private static final int WAIT_TIME_PER_RECORD = 10 * 1000; // 10 sec.
 
 		@Override
@@ -139,12 +141,12 @@ public class JoinCancelingITCase extends CancelingTestBase {
 			return new Tuple2<>(first.f0, second.f0);
 		}
 	}
-	
-	public static final class LongCancelTimeMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
+
+	private static final class LongCancelTimeMatcher<IN> implements JoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
 		private static final long serialVersionUID = 1L;
-		
+
 		private static final int WAIT_TIME_PER_RECORD = 5 * 1000; // 5 sec.
-		
+
 		@Override
 		public Tuple2<IN, IN> join(Tuple2<IN, IN> first, Tuple2<IN, IN> second) throws Exception {
 			final long start = System.currentTimeMillis();
@@ -157,10 +159,10 @@ public class JoinCancelingITCase extends CancelingTestBase {
 			return new Tuple2<>(first.f0, second.f0);
 		}
 	}
-	
-	public static final class StuckInOpenMatcher<IN> extends RichJoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
+
+	private static final class StuckInOpenMatcher<IN> extends RichJoinFunction<Tuple2<IN, IN>, Tuple2<IN, IN>, Tuple2<IN, IN>> {
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public void open(Configuration parameters) throws Exception {
 			synchronized (this) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
index 03b1a24..3a7039f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
@@ -24,30 +24,34 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.InfiniteIntegerInputFormat;
+
 import org.junit.Test;
 
+/**
+ * Test job cancellation from within a MapFunction.
+ */
 public class MapCancelingITCase extends CancelingTestBase {
 	private static final int parallelism = 4;
 
 	public MapCancelingITCase() {
 		setTaskManagerNumSlots(parallelism);
 	}
-	
+
 	@Test
 	public void testMapCancelling() throws Exception {
 		executeTask(new IdentityMapper<Integer>());
 	}
-	
+
 	@Test
 	public void testSlowMapCancelling() throws Exception {
 		executeTask(new DelayingIdentityMapper<Integer>());
 	}
-	
+
 	@Test
 	public void testMapWithLongCancellingResponse() throws Exception {
 		executeTask(new LongCancelTimeIdentityMapper<Integer>());
 	}
-	
+
 	@Test
 	public void testMapPriorToFirstRecordReading() throws Exception {
 		executeTask(new StuckInOpenIdentityMapper<Integer>());
@@ -67,8 +71,8 @@ public class MapCancelingITCase extends CancelingTestBase {
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
-	public static final class IdentityMapper<IN> implements MapFunction<IN, IN> {
+
+	private static final class IdentityMapper<IN> implements MapFunction<IN, IN> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -76,8 +80,8 @@ public class MapCancelingITCase extends CancelingTestBase {
 			return value;
 		}
 	}
-	
-	public static final class DelayingIdentityMapper<IN> implements MapFunction<IN, IN> {
+
+	private static final class DelayingIdentityMapper<IN> implements MapFunction<IN, IN> {
 		private static final long serialVersionUID = 1L;
 
 		private static final int WAIT_TIME_PER_VALUE = 10 * 1000; // 10 sec.
@@ -88,8 +92,8 @@ public class MapCancelingITCase extends CancelingTestBase {
 			return value;
 		}
 	}
-	
-	public static final class LongCancelTimeIdentityMapper<IN> implements MapFunction<IN, IN> {
+
+	private static final class LongCancelTimeIdentityMapper<IN> implements MapFunction<IN, IN> {
 		private static final long serialVersionUID = 1L;
 
 		private static final int WAIT_TIME_PER_VALUE = 5 * 1000; // 5 sec.
@@ -108,8 +112,8 @@ public class MapCancelingITCase extends CancelingTestBase {
 			return value;
 		}
 	}
-	
-	public static final class StuckInOpenIdentityMapper<IN> extends RichMapFunction<IN, IN> {
+
+	private static final class StuckInOpenIdentityMapper<IN> extends RichMapFunction<IN, IN> {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
index 0021b81..269b126 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java
@@ -47,6 +47,7 @@ import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -169,11 +170,11 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 	@Test
 	public void testTumblingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = numElementsPerKey();
-		final int WINDOW_SIZE = windowSize();
-		final int NUM_KEYS = numKeys();
+		final int numElementsPerKey = numElementsPerKey();
+		final int windowSize = windowSize();
+		final int numKeys = numKeys();
 		FailingSource.reset();
-		
+
 		try {
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -183,10 +184,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 			env.setStateBackend(this.stateBackend);
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
 					.keyBy(0)
-					.timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindow(Time.of(windowSize, MILLISECONDS))
 					.apply(new RichWindowFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, Tuple, TimeWindow>() {
 
 						private boolean open = false;
@@ -217,8 +218,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 							out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -239,9 +239,9 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 	}
 
 	public void doTestTumblingTimeWindowWithKVState(int maxParallelism) {
-		final int NUM_ELEMENTS_PER_KEY = numElementsPerKey();
-		final int WINDOW_SIZE = windowSize();
-		final int NUM_KEYS = numKeys();
+		final int numElementsPerKey = numElementsPerKey();
+		final int windowSize = windowSize();
+		final int numKeys = numKeys();
 		FailingSource.reset();
 
 		try {
@@ -254,10 +254,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 			env.setStateBackend(this.stateBackend);
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
 					.keyBy(0)
-					.timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindow(Time.of(windowSize, MILLISECONDS))
 					.apply(new RichWindowFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, Tuple, TimeWindow>() {
 
 						private boolean open = false;
@@ -292,8 +292,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 							out.collect(new Tuple4<>(tuple.<Long>getField(0), window.getStart(), window.getEnd(), new IntType(count.value())));
 						}
 					})
-					.addSink(new CountValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new CountValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -305,10 +304,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 	@Test
 	public void testSlidingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = numElementsPerKey();
-		final int WINDOW_SIZE = windowSize();
-		final int WINDOW_SLIDE = windowSlide();
-		final int NUM_KEYS = numKeys();
+		final int numElementsPerKey = numElementsPerKey();
+		final int windowSize = windowSize();
+		final int windowSlide = windowSlide();
+		final int numKeys = numKeys();
 		FailingSource.reset();
 
 		try {
@@ -322,10 +321,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 			env.getConfig().setUseSnapshotCompression(true);
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
 					.keyBy(0)
-					.timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS), Time.of(WINDOW_SLIDE, MILLISECONDS))
+					.timeWindow(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS))
 					.apply(new RichWindowFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, Tuple, TimeWindow>() {
 
 						private boolean open = false;
@@ -356,8 +355,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 							out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SLIDE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -369,9 +367,9 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 	@Test
 	public void testPreAggregatedTumblingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = numElementsPerKey();
-		final int WINDOW_SIZE = windowSize();
-		final int NUM_KEYS = numKeys();
+		final int numElementsPerKey = numElementsPerKey();
+		final int windowSize = windowSize();
+		final int numKeys = numKeys();
 		FailingSource.reset();
 
 		try {
@@ -383,10 +381,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 			env.setStateBackend(this.stateBackend);
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
 					.keyBy(0)
-					.timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindow(Time.of(windowSize, MILLISECONDS))
 					.reduce(
 							new ReduceFunction<Tuple2<Long, IntType>>() {
 
@@ -425,8 +423,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -438,10 +435,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 	@Test
 	public void testPreAggregatedSlidingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = numElementsPerKey();
-		final int WINDOW_SIZE = windowSize();
-		final int WINDOW_SLIDE = windowSlide();
-		final int NUM_KEYS = numKeys();
+		final int numElementsPerKey = numElementsPerKey();
+		final int windowSize = windowSize();
+		final int windowSlide = windowSlide();
+		final int numKeys = numKeys();
 		FailingSource.reset();
 
 		try {
@@ -453,10 +450,10 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 			env.setStateBackend(this.stateBackend);
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
 					.keyBy(0)
-					.timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS), Time.of(WINDOW_SLIDE, MILLISECONDS))
+					.timeWindow(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS))
 					.reduce(
 							new ReduceFunction<Tuple2<Long, IntType>>() {
 
@@ -497,8 +494,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SLIDE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -508,14 +504,12 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 		}
 	}
 
-
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements ListCheckpointed<Integer>, CheckpointListener
-	{
+			implements ListCheckpointed<Integer>, CheckpointListener {
 		private static volatile boolean failedBefore = false;
 
 		private final int numKeys;
@@ -556,8 +550,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 				}
 
 				if (numElementsEmitted < numElementsToEmit &&
-						(failedBefore || numElementsEmitted <= failureAfterNumElements))
-				{
+						(failedBefore || numElementsEmitted <= failureAfterNumElements)) {
 					// the function failed before, or we are in the elements before the failure
 					synchronized (ctx.getCheckpointLock()) {
 						int next = numElementsEmitted++;
@@ -669,7 +662,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 
 			assertEquals("Window start: " + value.f1 + " end: " + value.f2, expectedSum, value.f3.value);
 
-
 			Integer curr = windowCounts.get(value.f0);
 			if (curr != null) {
 				windowCounts.put(value.f0, curr + 1);
@@ -756,7 +748,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 				windowCounts.put(value.f0, 1);
 			}
 
-
 			// verify the contents of that window, the contents should be:
 			// (key + num windows so far)
 
@@ -799,13 +790,15 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	public static class IntType {
+	private static class IntType {
 
 		public int value;
 
 		public IntType() {}
 
-		public IntType(int value) { this.value = value; }
+		public IntType(int value) {
+			this.value = value;
+		}
 	}
 
 	protected int numElementsPerKey() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncFileBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncFileBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncFileBackendEventTimeWindowCheckpointingITCase.java
index a5bf10c..f0db4d5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncFileBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncFileBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for asynchronous file backend.
+ */
 public class AsyncFileBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public AsyncFileBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncMemBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncMemBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncMemBackendEventTimeWindowCheckpointingITCase.java
index ef9ad37..70ec757 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncMemBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AsyncMemBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for asynchronous memory backend.
+ */
 public class AsyncMemBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public AsyncMemBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
index 06d3ab0..d224905 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
@@ -50,12 +50,10 @@ import static org.junit.Assert.assertTrue;
  * {@link org.apache.flink.test.checkpointing.StreamCheckpointingITCase} in that it contains
  * a TwoInput (or co-) Task.
  *
- * <p>
- * This checks whether checkpoint barriers correctly trigger TwoInputTasks and also whether
+ * <p>This checks whether checkpoint barriers correctly trigger TwoInputTasks and also whether
  * this barriers are correctly forwarded.
  *
- * <p>
- * The test triggers a failure after a while and verifies that, after completion, the
+ * <p>The test triggers a failure after a while and verifies that, after completion, the
  * state reflects the "exactly once" semantics.
  */
 @SuppressWarnings({"serial", "deprecation"})
@@ -65,8 +63,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 	private static final int PARALLELISM = 4;
 
 	/**
-	 * Runs the following program:
-	 *
+	 * Runs the following program.
 	 * <pre>
 	 *     [ (source)->(filter)->(map) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
 	 * </pre>
@@ -136,7 +133,6 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		assertEquals(NUM_STRINGS, countSum);
 	}
 
-
 	// --------------------------------------------------------------------------------------------
 	//  Custom Functions
 	// --------------------------------------------------------------------------------------------
@@ -145,7 +141,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 	 * A generating source that is slow before the first two checkpoints went through
 	 * and will indefinitely stall at a certain point to allow the checkpoint to complete.
 	 *
-	 * After the checkpoints are through, it continues with full speed.
+	 * <p>After the checkpoints are through, it continues with full speed.
 	 */
 	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
 			implements ListCheckpointed<Integer>, CheckpointListener {
@@ -174,7 +170,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
 			if (index < 0) {
 				// not been restored, so initialize
-				index =getRuntimeContext().getIndexOfThisSubtask();
+				index = getRuntimeContext().getIndexOfThisSubtask();
 			}
 
 			while (isRunning && index < numElements) {
@@ -243,11 +239,11 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 	}
 
-	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
+	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount>
 			implements ListCheckpointed<Long> {
 
-		static final long[] counts = new long[PARALLELISM];
-		
+		static long[] counts = new long[PARALLELISM];
+
 		private long count;
 
 		@Override
@@ -312,7 +308,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 
 	private static class StringRichFilterFunction extends RichFilterFunction<String> implements ListCheckpointed<Long> {
 
-		static final long[] counts = new long[PARALLELISM];
+		static long[] counts = new long[PARALLELISM];
 
 		private long count = 0L;
 
@@ -342,8 +338,8 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 	}
 
 	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> implements ListCheckpointed<Long> {
-		
-		static final long[] counts = new long[PARALLELISM];
+
+		static long[] counts = new long[PARALLELISM];
 
 		private long count;
 
@@ -374,7 +370,7 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 
 	private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction<String, String, String> implements ListCheckpointed<Long> {
 
-		static final long[] counts = new long[PARALLELISM];
+		static long[] counts = new long[PARALLELISM];
 
 		private long count;
 
@@ -409,6 +405,9 @@ public class CoStreamCheckpointingITCase extends StreamingMultipleProgramsTestBa
 		}
 	}
 
+	/**
+	 * POJO storing a prefix, value, and count.
+	 */
 	public static class PrefixCount implements Serializable {
 
 		public String prefix;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
index 0ad42bb..280e11a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java
@@ -19,6 +19,7 @@
 package org.apache.flink.test.checkpointing;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.io.FilePathFilter;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.io.TextInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -30,10 +31,10 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
-import org.apache.flink.api.common.io.FilePathFilter;
 import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
+
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -56,6 +57,9 @@ import java.util.Set;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+/**
+ * Test checkpointing while sourcing a continuous file processor.
+ */
 public class ContinuousFileProcessingCheckpointITCase extends StreamFaultToleranceTestBase {
 
 	private static final int NO_OF_FILES = 5;
@@ -68,7 +72,7 @@ public class ContinuousFileProcessingCheckpointITCase extends StreamFaultToleran
 	private static String localFsURI;
 	private FileCreator fc;
 
-	private static  Map<Integer, Set<String>> actualCollectedContent = new HashMap<>();
+	private static Map<Integer, Set<String>> actualCollectedContent = new HashMap<>();
 
 	@BeforeClass
 	public static void createHDFS() {
@@ -78,10 +82,10 @@ public class ContinuousFileProcessingCheckpointITCase extends StreamFaultToleran
 
 			org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration();
 
-			localFsURI = "file:///" + baseDir +"/";
+			localFsURI = "file:///" + baseDir + "/";
 			localFs = new org.apache.hadoop.fs.Path(localFsURI).getFileSystem(hdConf);
 
-		} catch(Throwable e) {
+		} catch (Throwable e) {
 			e.printStackTrace();
 			Assert.fail("Test failed " + e.getMessage());
 		}
@@ -279,7 +283,7 @@ public class ContinuousFileProcessingCheckpointITCase extends StreamFaultToleran
 
 		public void run() {
 			try {
-				for(int i = 0; i < NO_OF_FILES; i++) {
+				for (int i = 0; i < NO_OF_FILES; i++) {
 					Tuple2<org.apache.hadoop.fs.Path, String> tmpFile;
 					long modTime;
 					do {
@@ -338,8 +342,8 @@ public class ContinuousFileProcessingCheckpointITCase extends StreamFaultToleran
 
 		FSDataOutputStream stream = localFs.create(tmp);
 		StringBuilder str = new StringBuilder();
-		for(int i = 0; i < LINES_PER_FILE; i++) {
-			String line = fileIdx +": "+ sampleLine + " " + i +"\n";
+		for (int i = 0; i < LINES_PER_FILE; i++) {
+			String line = fileIdx + ": " + sampleLine + " " + i + "\n";
 			str.append(line);
 			stream.write(line.getBytes(ConfigConstants.DEFAULT_CHARSET));
 		}


[17/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
index 6350533..add0cd1 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTerminationTail.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.List;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.java.DataSet;
@@ -27,6 +26,11 @@ import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 
+import java.util.List;
+
+/**
+ * Test iteration with termination criterion.
+ */
 public class IterationTerminationWithTerminationTail extends JavaProgramTestBase {
 	private static final String EXPECTED = "22\n";
 
@@ -48,7 +52,7 @@ public class IterationTerminationWithTerminationTail extends JavaProgramTestBase
 		containsResultAsText(result, EXPECTED);
 	}
 
-	public static final class SumReducer implements GroupReduceFunction<String, String> {
+	private static final class SumReducer implements GroupReduceFunction<String, String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -61,7 +65,7 @@ public class IterationTerminationWithTerminationTail extends JavaProgramTestBase
 		}
 	}
 
-	public static class TerminationFilter implements FilterFunction<String> {
+	private static class TerminationFilter implements FilterFunction<String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
index 5a2df3f..4f749b6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationTerminationWithTwoTails.java
@@ -18,16 +18,19 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.List;
 import org.apache.flink.api.common.functions.FilterFunction;
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 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.test.util.JavaProgramTestBase;
-import static org.apache.flink.test.util.TestBaseUtils.containsResultAsText;
 import org.apache.flink.util.Collector;
 
+import java.util.List;
+
+/**
+ * Test iteration with termination criterion consuming the iteration tail.
+ */
 public class IterationTerminationWithTwoTails extends JavaProgramTestBase {
 	private static final String EXPECTED = "22\n";
 
@@ -49,7 +52,7 @@ public class IterationTerminationWithTwoTails extends JavaProgramTestBase {
 		containsResultAsText(result, EXPECTED);
 	}
 
-	public static final class SumReducer implements GroupReduceFunction<String, String> {
+	private static final class SumReducer implements GroupReduceFunction<String, String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -62,7 +65,7 @@ public class IterationTerminationWithTwoTails extends JavaProgramTestBase {
 		}
 	}
 
-	public static class TerminationFilter implements FilterFunction<String> {
+	private static class TerminationFilter implements FilterFunction<String> {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
index ab66f31..3228b73 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithAllReducerITCase.java
@@ -18,13 +18,17 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.List;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.util.List;
+
+/**
+ * Test iterator with an all-reduce.
+ */
 public class IterationWithAllReducerITCase extends JavaProgramTestBase {
 	private static final String EXPECTED = "1\n";
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
index c283df1..2ba1a8f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithChainingITCase.java
@@ -24,12 +24,15 @@ 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.test.util.PointFormatter;
-import org.apache.flink.test.util.PointInFormat;
 import org.apache.flink.test.util.CoordVector;
 import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.test.util.PointFormatter;
+import org.apache.flink.test.util.PointInFormat;
 import org.apache.flink.util.Collector;
 
+/**
+ * Test iteration with operator chaining.
+ */
 public class IterationWithChainingITCase extends JavaProgramTestBase {
 
 	private static final String DATA_POINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n";
@@ -48,13 +51,13 @@ public class IterationWithChainingITCase extends JavaProgramTestBase {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(4);
 
-		DataSet<Tuple2<Integer, CoordVector>> initialInput
-				= env.readFile(new PointInFormat(), dataPath).setParallelism(1).name("Input");
+		DataSet<Tuple2<Integer, CoordVector>> initialInput =
+				env.readFile(new PointInFormat(), dataPath).setParallelism(1).name("Input");
 
 		IterativeDataSet<Tuple2<Integer, CoordVector>> iteration = initialInput.iterate(2).name("Loop");
 
-		DataSet<Tuple2<Integer, CoordVector>> identity
-				= iteration.groupBy(0).reduceGroup(new GroupReduceFunction<Tuple2<Integer, CoordVector>, Tuple2<Integer, CoordVector>>() {
+		DataSet<Tuple2<Integer, CoordVector>> identity =
+				iteration.groupBy(0).reduceGroup(new GroupReduceFunction<Tuple2<Integer, CoordVector>, Tuple2<Integer, CoordVector>>() {
 					@Override
 					public void reduce(Iterable<Tuple2<Integer, CoordVector>> values, Collector<Tuple2<Integer, CoordVector>> out) throws Exception {
 						for (Tuple2<Integer, CoordVector> value : values) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
index 8756429..e44d870 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationWithUnionITCase.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import java.io.Serializable;
-
 import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
@@ -27,11 +25,16 @@ 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.test.util.CoordVector;
+import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.test.util.PointFormatter;
 import org.apache.flink.test.util.PointInFormat;
-import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 
+import java.io.Serializable;
+
+/**
+ * Test iteration with union.
+ */
 public class IterationWithUnionITCase extends JavaProgramTestBase {
 
 	private static final String DATAPOINTS = "0|50.90|16.20|72.08|\n" + "1|73.65|61.76|62.89|\n" + "2|61.73|49.95|92.74|\n";
@@ -39,13 +42,12 @@ public class IterationWithUnionITCase extends JavaProgramTestBase {
 	protected String dataPath;
 	protected String resultPath;
 
-
 	@Override
 	protected void preSubmit() throws Exception {
 		dataPath = createTempFile("datapoints.txt", DATAPOINTS);
 		resultPath = getTempDirPath("union_iter_result");
 	}
-	
+
 	@Override
 	protected void postSubmit() throws Exception {
 		compareResultsByLinesInMemory(DATAPOINTS + DATAPOINTS + DATAPOINTS + DATAPOINTS, resultPath);
@@ -54,18 +56,18 @@ public class IterationWithUnionITCase extends JavaProgramTestBase {
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		DataSet<Tuple2<Integer, CoordVector>> initialInput = env.readFile(new PointInFormat(), this.dataPath).setParallelism(1);
-		
+
 		IterativeDataSet<Tuple2<Integer, CoordVector>> iteration = initialInput.iterate(2);
-		
+
 		DataSet<Tuple2<Integer, CoordVector>> result = iteration.union(iteration).map(new IdentityMapper());
-		
+
 		iteration.closeWith(result).writeAsFormattedText(this.resultPath, new PointFormatter());
-		
+
 		env.execute();
 	}
-	
+
 	static final class IdentityMapper implements MapFunction<Tuple2<Integer, CoordVector>, Tuple2<Integer, CoordVector>>, Serializable {
 		private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java
index 7bd9934..77e2663 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/KMeansWithBroadcastSetITCase.java
@@ -23,14 +23,17 @@ 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.examples.java.clustering.KMeans;
-import org.apache.flink.examples.java.clustering.KMeans.Point;
 import org.apache.flink.examples.java.clustering.KMeans.Centroid;
-import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.examples.java.clustering.KMeans.Point;
 import org.apache.flink.test.testdata.KMeansData;
+import org.apache.flink.test.util.JavaProgramTestBase;
 
 import java.util.List;
 import java.util.Locale;
 
+/**
+ * Test KMeans clustering with a broadcast set.
+ */
 public class KMeansWithBroadcastSetITCase extends JavaProgramTestBase {
 
 	@SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java
index e6e91f6..cc3fdc6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/MultipleSolutionSetJoinsITCase.java
@@ -18,41 +18,44 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.junit.Assert;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.optimizer.iterations.MultipleJoinsWithSolutionSetCompilerTest;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import org.junit.Assert;
+
+import java.util.ArrayList;
+import java.util.List;
 
+/**
+ * Test multiple joins with the solution set.
+ */
 public class MultipleSolutionSetJoinsITCase extends JavaProgramTestBase {
 
 	@Override
 	protected void testProgram() throws Exception {
-		
-		final int NUM_ITERS = 4;
-		final double expectedFactor = (int) Math.pow(7, NUM_ITERS);
-		
+
+		final int numIters = 4;
+		final double expectedFactor = (int) Math.pow(7, numIters);
+
 		// this is an artificial program, it does not compute anything sensical
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		@SuppressWarnings("unchecked")
 		DataSet<Tuple2<Long, Double>> initialData = env.fromElements(new Tuple2<Long, Double>(1L, 1.0), new Tuple2<Long, Double>(2L, 2.0),
 															new Tuple2<Long, Double>(3L, 3.0), new Tuple2<Long, Double>(4L, 4.0),
 															new Tuple2<Long, Double>(5L, 5.0), new Tuple2<Long, Double>(6L, 6.0));
-		
-		DataSet<Tuple2<Long, Double>> result = MultipleJoinsWithSolutionSetCompilerTest.constructPlan(initialData, NUM_ITERS);
-		
-		List<Tuple2<Long, Double>> resultCollector = new ArrayList<Tuple2<Long,Double>>();
-		result.output(new LocalCollectionOutputFormat<Tuple2<Long,Double>>(resultCollector));
-		
+
+		DataSet<Tuple2<Long, Double>> result = MultipleJoinsWithSolutionSetCompilerTest.constructPlan(initialData, numIters);
+
+		List<Tuple2<Long, Double>> resultCollector = new ArrayList<Tuple2<Long, Double>>();
+		result.output(new LocalCollectionOutputFormat<>(resultCollector));
+
 		env.execute();
-		
+
 		for (Tuple2<Long, Double> tuple : resultCollector) {
 			Assert.assertEquals(expectedFactor * tuple.f0, tuple.f1.doubleValue(), 0.0);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java
index c987dfd..a402747 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/SolutionSetDuplicatesITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -37,6 +38,9 @@ import java.util.List;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+/**
+ * Test for duplicate elimination in the solution set.
+ */
 @SuppressWarnings("serial")
 @RunWith(Parameterized.class)
 public class SolutionSetDuplicatesITCase extends MultipleProgramsTestBase {
@@ -66,7 +70,7 @@ public class SolutionSetDuplicatesITCase extends MultipleProgramsTestBase {
 
 			List<Integer> result = iter
 					.closeWith(iter.getWorkset(), iter.getWorkset())
-					.map(new MapFunction<Tuple2<Long,Long>, Integer>() {
+					.map(new MapFunction<Tuple2<Long, Long>, Integer>() {
 						@Override
 						public Integer map(Tuple2<Long, Long> value) {
 							return value.f0.intValue();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java
index 766a422..0228aef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/StaticlyNestedIterationsITCase.java
@@ -27,34 +27,35 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+/**
+ * Test iterations referenced from the static path of other iterations.
+ */
 public class StaticlyNestedIterationsITCase extends JavaProgramTestBase {
 
-	
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		DataSet<Long> data1 = env.generateSequence(1, 100);
 		DataSet<Long> data2 = env.generateSequence(1, 100);
-		
+
 		IterativeDataSet<Long> firstIteration = data1.iterate(100);
-		
+
 		DataSet<Long> firstResult = firstIteration.closeWith(firstIteration.map(new IdMapper()));
-		
-		
+
 		IterativeDataSet<Long> mainIteration = data2.map(new IdMapper()).iterate(100);
-		
+
 		DataSet<Long> joined = mainIteration.join(firstResult)
 				.where(new IdKeyExtractor()).equalTo(new IdKeyExtractor())
 				.with(new Joiner());
-		
+
 		DataSet<Long> mainResult = mainIteration.closeWith(joined);
-		
+
 		mainResult.output(new DiscardingOutputFormat<Long>());
-		
+
 		env.execute();
 	}
-	
+
 	private static class IdKeyExtractor implements KeySelector<Long, Long> {
 
 		private static final long serialVersionUID = 1L;
@@ -64,21 +65,21 @@ public class StaticlyNestedIterationsITCase extends JavaProgramTestBase {
 			return value;
 		}
 	}
-	
+
 	private static class IdMapper implements MapFunction<Long, Long> {
-		
+
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Long map(Long value) {
 			return value;
 		}
 	}
-	
+
 	private static class Joiner implements JoinFunction<Long, Long, Long> {
-		
+
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Long join(Long first, Long second) {
 			return first;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java
index fa8643f..01e578c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/UnionStaticDynamicIterationITCase.java
@@ -18,36 +18,39 @@
 
 package org.apache.flink.test.iterative;
 
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.util.ArrayList;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test union between static and dynamic path in an iteration.
+ */
 public class UnionStaticDynamicIterationITCase  extends JavaProgramTestBase {
-	
+
 	private final ArrayList<Long> result = new ArrayList<Long>();
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		DataSet<Long> inputStatic = env.generateSequence(1, 4);
 		DataSet<Long> inputIteration = env.generateSequence(1, 4);
-		
+
 		IterativeDataSet<Long> iteration = inputIteration.iterate(3);
-		
+
 		DataSet<Long> result = iteration.closeWith(inputStatic.union(inputStatic).union(iteration.union(iteration)));
-			
+
 		result.output(new LocalCollectionOutputFormat<Long>(this.result));
-		
+
 		env.execute();
 	}
-	
+
 	@Override
 	protected void postSubmit() throws Exception {
 		assertEquals(88, result.size());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/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
index 3bced25..2403cd9 100644
--- 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
@@ -18,33 +18,33 @@
 
 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.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.operators.DeltaIteration;
+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.test.util.MultipleProgramsTestBase;
 import org.apache.flink.test.util.TestBaseUtils;
 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.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
 
 /**
- * Connected Components test case that uses a parameterizable convergence criterion
+ * Connected Components test case that uses a parameterizable convergence criterion.
  */
 @RunWith(Parameterized.class)
 @SuppressWarnings("serial")
@@ -55,181 +55,181 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 	}
 
 	final List<Tuple2<Long, Long>> verticesInput = Arrays.asList(
-			new Tuple2<>(1l,1l),
-			new Tuple2<>(2l,2l),
-			new Tuple2<>(3l,3l),
-			new Tuple2<>(4l,4l),
-			new Tuple2<>(5l,5l),
-			new Tuple2<>(6l,6l),
-			new Tuple2<>(7l,7l),
-			new Tuple2<>(8l,8l),
-			new Tuple2<>(9l,9l)
+		new Tuple2<>(1L, 1L),
+		new Tuple2<>(2L, 2L),
+		new Tuple2<>(3L, 3L),
+		new Tuple2<>(4L, 4L),
+		new Tuple2<>(5L, 5L),
+		new Tuple2<>(6L, 6L),
+		new Tuple2<>(7L, 7L),
+		new Tuple2<>(8L, 8L),
+		new Tuple2<>(9L, 9L)
 	);
 
 	final 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)
+		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 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,2L),
-			new Tuple2<>(6L,1L),
-			new Tuple2<>(7L,7L),
-			new Tuple2<>(8L,7L),
-			new Tuple2<>(9L,7L)
+		new Tuple2<>(1L, 1L),
+		new Tuple2<>(2L, 1L),
+		new Tuple2<>(3L, 1L),
+		new Tuple2<>(4L, 1L),
+		new Tuple2<>(5L, 2L),
+		new Tuple2<>(6L, 1L),
+		new Tuple2<>(7L, 7L),
+		new Tuple2<>(8L, 7L),
+		new Tuple2<>(9L, 7L)
 	);
 
 	@Test
 	public void testConnectedComponentsWithParametrizableConvergence() throws Exception {
 
-			// name of the aggregator that checks for convergence
-			final String UPDATED_ELEMENTS = "updated.elements.aggr";
+		// name of the aggregator that checks for convergence
+		final String updatedElements = "updated.elements.aggr";
+
+		// the iteration stops if less than this number of elements change value
+		final long convergenceThreshold = 3;
 
-			// the iteration stops if less than this number of elements change value
-			final long convergence_threshold = 3;
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
-			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
+		DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
 
-			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
-			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
+		IterativeDataSet<Tuple2<Long, Long>> iteration = initialSolutionSet.iterate(10);
 
-			IterativeDataSet<Tuple2<Long, Long>> iteration = initialSolutionSet.iterate(10);
+		// register the convergence criterion
+		iteration.registerAggregationConvergenceCriterion(updatedElements,
+			new LongSumAggregator(), new UpdatedElementsConvergenceCriterion(convergenceThreshold));
 
-			// 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>> 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(updatedElements));
 
-			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 TestBaseUtils.TupleComparator<Tuple2<Long, Long>>());
 
-			List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId).collect();
-			Collections.sort(result, new TestBaseUtils.TupleComparator<Tuple2<Long, Long>>());
-			
-			assertEquals(expectedResult, result);
+		assertEquals(expectedResult, result);
 	}
 
 	@Test
 	public void testDeltaConnectedComponentsWithParametrizableConvergence() throws Exception {
 
-			// name of the aggregator that checks for convergence
-			final String UPDATED_ELEMENTS = "updated.elements.aggr";
+		// name of the aggregator that checks for convergence
+		final String updatedElements = "updated.elements.aggr";
 
-			// the iteration stops if less than this number of elements change value
-			final long convergence_threshold = 3;
+		// the iteration stops if less than this number of elements change value
+		final long convergenceThreshold = 3;
 
-			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
-			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
-			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
+		DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
+		DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
 
-			DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
-					initialSolutionSet.iterateDelta(initialSolutionSet, 10, 0);
+		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+			initialSolutionSet.iterateDelta(initialSolutionSet, 10, 0);
 
-			// register the convergence criterion
-			iteration.registerAggregationConvergenceCriterion(UPDATED_ELEMENTS,
-					new LongSumAggregator(), new UpdatedElementsConvergenceCriterion(convergence_threshold));
+		// register the convergence criterion
+		iteration.registerAggregationConvergenceCriterion(updatedElements,
+			new LongSumAggregator(), new UpdatedElementsConvergenceCriterion(convergenceThreshold));
 
-			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.getWorkset().join(edges).where(0).equalTo(0)
-					.with(new NeighborWithComponentIDJoin())
-					.groupBy(0).min(1);
+		DataSet<Tuple2<Long, Long>> verticesWithNewComponents = iteration.getWorkset().join(edges).where(0).equalTo(0)
+			.with(new NeighborWithComponentIDJoin())
+			.groupBy(0).min(1);
 
-			DataSet<Tuple2<Long, Long>> updatedComponentId =
-					verticesWithNewComponents.join(iteration.getSolutionSet()).where(0).equalTo(0)
-							.flatMap(new MinimumIdFilter(UPDATED_ELEMENTS));
+		DataSet<Tuple2<Long, Long>> updatedComponentId =
+			verticesWithNewComponents.join(iteration.getSolutionSet()).where(0).equalTo(0)
+				.flatMap(new MinimumIdFilter(updatedElements));
 
-			List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId, updatedComponentId).collect();
-			Collections.sort(result, new TestBaseUtils.TupleComparator<Tuple2<Long, Long>>());
+		List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId, updatedComponentId).collect();
+		Collections.sort(result, new TestBaseUtils.TupleComparator<Tuple2<Long, Long>>());
 
-			assertEquals(expectedResult, result);
+		assertEquals(expectedResult, result);
 	}
-	
+
 	@Test
 	public void testParameterizableAggregator() throws Exception {
 
-			final int MAX_ITERATIONS = 5;
-			final String AGGREGATOR_NAME = "elements.in.component.aggregator";
-			final long componentId = 1l;
+			final int maxIterations = 5;
+		final String aggregatorName = "elements.in.component.aggregator";
+		final long componentId = 1L;
 
-			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
-			DataSet<Tuple2<Long, Long>> initialSolutionSet = env.fromCollection(verticesInput);
-			DataSet<Tuple2<Long, Long>> edges = env.fromCollection(edgesInput);
+		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);
+		IterativeDataSet<Tuple2<Long, Long>> iteration =
+			initialSolutionSet.iterate(maxIterations);
 
-			// register the aggregator
-			iteration.registerAggregator(AGGREGATOR_NAME, new LongSumAggregatorWithParameter(componentId));
+		// register the aggregator
+		iteration.registerAggregator(aggregatorName, 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>> 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));
+		DataSet<Tuple2<Long, Long>> updatedComponentId =
+			verticesWithNewComponents.join(iteration).where(0).equalTo(0)
+				.flatMap(new MinimumIdFilterCounting(aggregatorName));
 
-			List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId).collect();
+		List<Tuple2<Long, Long>> result = iteration.closeWith(updatedComponentId).collect();
 
-			Collections.sort(result, new TestBaseUtils.TupleComparator<Tuple2<Long, Long>>());
+		Collections.sort(result, new TestBaseUtils.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)
-			);
+		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)
+		);
 
-			// check program result
-			assertEquals(expectedResult, result);
+		// check program result
+		assertEquals(expectedResult, result);
 
-			// check aggregators
-			long[] aggr_values = MinimumIdFilterCounting.aggr_value;
+		// check aggregators
+		long[] aggrValues = 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]);
+		// 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, aggrValues[0]);
+		assertEquals(4, aggrValues[1]);
+		assertEquals(5, aggrValues[2]);
+		assertEquals(6, aggrValues[3]);
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Test Functions
 	// ------------------------------------------------------------------------
 
-	public static final class NeighborWithComponentIDJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static final class NeighborWithComponentIDJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -239,8 +239,8 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 			return vertexWithCompId;
 		}
 	}
-	
-	public static class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
+
+	private static class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
 
 		private final String aggName;
 		private LongSumAggregator aggr;
@@ -261,7 +261,7 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 
 			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
 				out.collect(vertexWithNewAndOldId.f0);
-				aggr.aggregate(1l);
+				aggr.aggregate(1L);
 			}
 			else {
 				out.collect(vertexWithNewAndOldId.f1);
@@ -269,24 +269,24 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 		}
 	}
 
-	public static final class MinimumIdFilterCounting 
+	private 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();
@@ -301,24 +301,26 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
 				out.collect(vertexWithNewAndOldId.f0);
 				if (vertexWithNewAndOldId.f0.f1 == aggr.getComponentId()) {
-					aggr.aggregate(1l);
+					aggr.aggregate(1L);
 				}
 			} else {
 				out.collect(vertexWithNewAndOldId.f1);
 				if (vertexWithNewAndOldId.f1.f1 == aggr.getComponentId()) {
-					aggr.aggregate(1l);
+					aggr.aggregate(1L);
 				}
 			}
 		}
 	}
 
-	/** A Convergence Criterion with one parameter */
-	public static class UpdatedElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
+	/**
+	 * A Convergence Criterion with one parameter.
+	 */
+	private static class UpdatedElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
 
 		private final long threshold;
 
-		public UpdatedElementsConvergenceCriterion(long u_threshold) {
-			this.threshold = u_threshold;
+		public UpdatedElementsConvergenceCriterion(long uThreshold) {
+			this.threshold = uThreshold;
 		}
 
 		@Override
@@ -327,7 +329,7 @@ public class AggregatorConvergenceITCase extends MultipleProgramsTestBase {
 		}
 	}
 
-	public static final class LongSumAggregatorWithParameter extends LongSumAggregator {
+	private static final class LongSumAggregatorWithParameter extends LongSumAggregator {
 
 		private long componentId;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
index 042617d..64ee98a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java
@@ -18,38 +18,38 @@
 
 package org.apache.flink.test.iterative.aggregators;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.util.Random;
-
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.After;
-import org.junit.Assert;
-
 import org.apache.flink.api.common.aggregators.ConvergenceCriterion;
 import org.apache.flink.api.common.aggregators.LongSumAggregator;
+import org.apache.flink.api.common.functions.RichFilterFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.DeltaIteration;
+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.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.util.Collector;
+
+import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.util.Random;
+
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -99,7 +99,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		env.registerCachedFile(resultPath, testName);
 
 		IterativeDataSet<Long> solution = env.fromElements(1L).iterate(2);
-		solution.closeWith(env.generateSequence(1,2).filter(new RichFilterFunction<Long>() {
+		solution.closeWith(env.generateSequence(1, 2).filter(new RichFilterFunction<Long>() {
 			@Override
 			public void open(Configuration parameters) throws Exception{
 				File file = getRuntimeContext().getDistributedCache().getFile(testName);
@@ -108,6 +108,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 				reader.close();
 				assertEquals(output, testString);
 			}
+
 			@Override
 			public boolean filter(Long value) throws Exception {
 				return false;
@@ -311,7 +312,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class NegativeElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
+	private static final class NegativeElementsConvergenceCriterion implements ConvergenceCriterion<LongValue> {
 
 		@Override
 		public boolean isConverged(int iteration, LongValue value) {
@@ -320,7 +321,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class NegativeElementsConvergenceCriterionWithParam implements ConvergenceCriterion<LongValue> {
+	private static final class NegativeElementsConvergenceCriterionWithParam implements ConvergenceCriterion<LongValue> {
 
 		private int value;
 
@@ -339,7 +340,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class SubtractOneMap extends RichMapFunction<Integer, Integer> {
+	private static final class SubtractOneMap extends RichMapFunction<Integer, Integer> {
 
 		private LongSumAggregator aggr;
 
@@ -354,14 +355,14 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 			Integer newValue = value - 1;
 			// count negative numbers
 			if (newValue < 0) {
-				aggr.aggregate(1l);
+				aggr.aggregate(1L);
 			}
 			return newValue;
 		}
 	}
 
 	@SuppressWarnings("serial")
-	public static final class SubtractOneMapWithParam extends RichMapFunction<Integer, Integer> {
+	private static final class SubtractOneMapWithParam extends RichMapFunction<Integer, Integer> {
 
 		private LongSumAggregatorWithParameter aggr;
 
@@ -374,15 +375,15 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		public Integer map(Integer value) {
 			Integer newValue = value - 1;
 			// count numbers less than the aggregator parameter
-			if ( newValue < aggr.getValue() ) {
-				aggr.aggregate(1l);
+			if (newValue < aggr.getValue()) {
+				aggr.aggregate(1L);
 			}
 			return newValue;
 		}
 	}
 
 	@SuppressWarnings("serial")
-	public static class LongSumAggregatorWithParameter extends LongSumAggregator {
+	private static class LongSumAggregatorWithParameter extends LongSumAggregator {
 
 		private int value;
 
@@ -396,7 +397,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class TupleMakerMap extends RichMapFunction<Integer, Tuple2<Integer, Integer>> {
+	private static final class TupleMakerMap extends RichMapFunction<Integer, Tuple2<Integer, Integer>> {
 
 		private Random rnd;
 
@@ -414,7 +415,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class AggregateMapDelta extends RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
+	private static final class AggregateMapDelta extends RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
 
 		private LongSumAggregator aggr;
 		private LongValue previousAggr;
@@ -437,14 +438,14 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
 			// count the elements that are equal to the superstep number
 			if (value.f1 == superstep) {
-				aggr.aggregate(1l);
+				aggr.aggregate(1L);
 			}
 			return value;
 		}
 	}
 
 	@SuppressWarnings("serial")
-	public static final class UpdateFilter extends RichFlatMapFunction<Tuple2<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>,
+	private static final class UpdateFilter extends RichFlatMapFunction<Tuple2<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>,
 			Tuple2<Integer, Integer>> {
 
 		private int superstep;
@@ -465,7 +466,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class ProjectSecondMapper extends RichMapFunction<Tuple2<Integer, Integer>, Integer> {
+	private static final class ProjectSecondMapper extends RichMapFunction<Tuple2<Integer, Integer>, Integer> {
 
 		@Override
 		public Integer map(Tuple2<Integer, Integer> value) {
@@ -474,7 +475,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static final class AggregateAndSubtractOneDelta extends RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
+	private static final class AggregateAndSubtractOneDelta extends RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> {
 
 		private LongSumAggregator aggr;
 		private LongValue previousAggr;
@@ -497,7 +498,7 @@ public class AggregatorsITCase extends MultipleProgramsTestBase {
 		public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) {
 			// count the ones
 			if (value.f1 == 1) {
-				aggr.aggregate(1l);
+				aggr.aggregate(1L);
 			}
 			value.f1--;
 			return value;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
deleted file mode 100644
index fc01ce7..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/AggregateITCase.java
+++ /dev/null
@@ -1,180 +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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.ValueCollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.StringValue;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class AggregateITCase extends MultipleProgramsTestBase {
-
-
-	public AggregateITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Test
-	public void testFullAggregate() throws Exception {
-		/*
-		 * Full Aggregate
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> aggregateDs = ds
-				.aggregate(Aggregations.SUM, 0)
-				.and(Aggregations.MAX, 1)
-				.project(0, 1);
-
-		List<Tuple2<Integer, Long>> result = aggregateDs.collect();
-
-		String expected = "231,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testFullAggregateOfMutableValueTypes() throws Exception {
-		/*
-		 * Full Aggregate of mutable value types
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds
-				.aggregate(Aggregations.SUM, 0)
-				.and(Aggregations.MAX, 1)
-				.project(0, 1);
-
-		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
-
-		String expected = "231,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testGroupedAggregate() throws Exception {
-		/*
-		 * Grouped Aggregate
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Long, Integer>> aggregateDs = ds.groupBy(1)
-				.aggregate(Aggregations.SUM, 0)
-				.project(1, 0);
-
-		List<Tuple2<Long, Integer>> result = aggregateDs.collect();
-
-		String expected = "1,1\n" +
-				"2,5\n" +
-				"3,15\n" +
-				"4,34\n" +
-				"5,65\n" +
-				"6,111\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testGroupedAggregateOfMutableValueTypes() throws Exception {
-		/*
-		 * Grouped Aggregate of mutable value types
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<IntValue, LongValue>> aggregateDs = ds.groupBy(1)
-				.aggregate(Aggregations.SUM, 0)
-				.project(1, 0);
-
-		List<Tuple2<IntValue, LongValue>> result = aggregateDs.collect();
-
-		String expected = "1,1\n" +
-				"2,5\n" +
-				"3,15\n" +
-				"4,34\n" +
-				"5,65\n" +
-				"6,111\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedAggregate() throws Exception {
-		/*
-		 * Nested Aggregate
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple1<Integer>> aggregateDs = ds.groupBy(1)
-				.aggregate(Aggregations.MIN, 0)
-				.aggregate(Aggregations.MIN, 0)
-				.project(0);
-
-		List<Tuple1<Integer>> result = aggregateDs.collect();
-
-		String expected = "1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedAggregateOfMutableValueTypes() throws Exception {
-		/*
-		 * Nested Aggregate of mutable value types
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<IntValue, LongValue, StringValue>> ds = ValueCollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple1<IntValue>> aggregateDs = ds.groupBy(1)
-				.aggregate(Aggregations.MIN, 0)
-				.aggregate(Aggregations.MIN, 0)
-				.project(0);
-
-		List<Tuple1<IntValue>> result = aggregateDs.collect();
-
-		String expected = "1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java
deleted file mode 100644
index 35d615b..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupGroupSortITCase.java
+++ /dev/null
@@ -1,122 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-
-@SuppressWarnings({"serial", "unchecked"})
-public class CoGroupGroupSortITCase extends JavaProgramTestBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
-		
-		DataSet<Tuple2<Long, Long>> input1 = env.fromElements(
-				new Tuple2<Long, Long>(0L, 5L),
-				new Tuple2<Long, Long>(0L, 4L),
-				new Tuple2<Long, Long>(0L, 3L),
-				new Tuple2<Long, Long>(0L, 2L),
-				new Tuple2<Long, Long>(0L, 1L),
-				new Tuple2<Long, Long>(1L, 10L),
-				new Tuple2<Long, Long>(1L, 8L),
-				new Tuple2<Long, Long>(1L, 9L),
-				new Tuple2<Long, Long>(1L, 7L));
-		
-		DataSet<TestPojo> input2 = env.fromElements(
-				new TestPojo(0L, 10L, 3L),
-				new TestPojo(0L, 8L, 3L),
-				new TestPojo(0L, 10L, 1L),
-				new TestPojo(0L, 9L, 0L),
-				new TestPojo(0L, 8L, 2L),
-				new TestPojo(0L, 8L, 4L),
-				new TestPojo(1L, 10L, 3L),
-				new TestPojo(1L, 8L, 3L),
-				new TestPojo(1L, 10L, 1L),
-				new TestPojo(1L, 9L, 0L),
-				new TestPojo(1L, 8L, 2L),
-				new TestPojo(1L, 8L, 4L));
-		
-		input1.coGroup(input2)
-		.where(1).equalTo("b")
-		.sortFirstGroup(0, Order.DESCENDING)
-		.sortSecondGroup("c", Order.ASCENDING).sortSecondGroup("a", Order.DESCENDING)
-		
-		.with(new ValidatingCoGroup())
-		.output(new DiscardingOutputFormat<NullValue>());
-		
-		env.execute();
-	}
-	
-	
-	private static class ValidatingCoGroup implements CoGroupFunction<Tuple2<Long, Long>, TestPojo, NullValue> {
-
-		@Override
-		public void coGroup(Iterable<Tuple2<Long, Long>> first, Iterable<TestPojo> second, Collector<NullValue> out) throws Exception {
-			// validate the tuple input, field 1, descending
-			{
-				long lastValue = Long.MAX_VALUE;
-				
-				for (Tuple2<Long, Long> t : first) {
-					long current = t.f1;
-					Assert.assertTrue(current <= lastValue);
-					lastValue = current;
-				}
-			}
-			
-			
-			// validate the pojo input
-			{
-				TestPojo lastValue = new TestPojo(Long.MAX_VALUE, 0, Long.MIN_VALUE);
-				
-				for (TestPojo current : second) {
-					Assert.assertTrue(current.c >= lastValue.c);
-					Assert.assertTrue(current.c != lastValue.c || current.a <= lastValue.a);
-					
-					lastValue = current;
-				}
-			}
-			
-		}
-	}
-	
-	public static class TestPojo implements Cloneable {
-		public long a;
-		public long b;
-		public long c;
-		
-		
-		public TestPojo() {}
-		
-		public TestPojo(long a, long b, long c) {
-			this.a = a;
-			this.b = b;
-			this.c = c;
-		}
-	}
-}


[15/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
deleted file mode 100644
index d62bcd4..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.javaApiOperators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.api.common.io.GenericInputFormat;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.util.Collector;
-
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test ExecutionEnvironment from user perspective
- */
-@SuppressWarnings("serial")
-public class ExecutionEnvironmentITCase extends TestLogger {
-	
-	private static final int PARALLELISM = 5;
-
-	/**
-	 * Ensure that the user can pass a custom configuration object to the LocalEnvironment
-	 */
-	@Test
-	public void testLocalEnvironmentWithConfig() throws Exception {
-		Configuration conf = new Configuration();
-		conf.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-		
-		final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(conf);
-		env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
-		env.getConfig().disableSysoutLogging();
-
-		DataSet<Integer> result = env.createInput(new ParallelismDependentInputFormat())
-				.rebalance()
-				.mapPartition(new RichMapPartitionFunction<Integer, Integer>() {
-					@Override
-					public void mapPartition(Iterable<Integer> values, Collector<Integer> out) throws Exception {
-						out.collect(getRuntimeContext().getIndexOfThisSubtask());
-					}
-				});
-		List<Integer> resultCollection = result.collect();
-		assertEquals(PARALLELISM, resultCollection.size());
-	}
-
-	private static class ParallelismDependentInputFormat extends GenericInputFormat<Integer> {
-
-		private transient boolean emitted;
-
-		@Override
-		public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
-			assertEquals(PARALLELISM, numSplits);
-			return super.createInputSplits(numSplits);
-		}
-
-		@Override
-		public boolean reachedEnd() {
-			return emitted;
-		}
-
-		@Override
-		public Integer nextRecord(Integer reuse) {
-			if (emitted) {
-				return null;
-			}
-			emitted = true;
-			return 1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
deleted file mode 100644
index 993b137..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FilterITCase.java
+++ /dev/null
@@ -1,327 +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.javaApiOperators;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class FilterITCase extends MultipleProgramsTestBase {
-	public FilterITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testAllRejectingFilter() throws Exception {
-		/*
-		 * Test all-rejecting filter.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new Filter1());
-
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Filter1 implements FilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return false;
-		}
-	}
-
-	@Test
-	public void testAllPassingFilter() throws Exception {
-		/*
-		 * Test all-passing filter.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new Filter2());
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n" +
-				"6,3,Luke Skywalker\n" +
-				"7,4,Comment#1\n" +
-				"8,4,Comment#2\n" +
-				"9,4,Comment#3\n" +
-				"10,4,Comment#4\n" +
-				"11,5,Comment#5\n" +
-				"12,5,Comment#6\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" +
-				"15,5,Comment#9\n" +
-				"16,6,Comment#10\n" +
-				"17,6,Comment#11\n" +
-				"18,6,Comment#12\n" +
-				"19,6,Comment#13\n" +
-				"20,6,Comment#14\n" +
-				"21,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Filter2 implements FilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return true;
-		}
-	}
-
-	@Test
-	public void testFilterOnStringTupleField() throws Exception {
-		/*
-		 * Test filter on String tuple field.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new Filter3());
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "3,2,Hello world\n"
-				+
-				"4,3,Hello world, how are you?\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	public static class Filter3 implements FilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return value.f2.contains("world");
-		}
-	}
-
-	@Test
-	public void testFilterOnIntegerTupleField() throws Exception {
-		/*
-		 * Test filter on Integer tuple field.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new Filter4());
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "2,2,Hello\n" +
-				"4,3,Hello world, how are you?\n" +
-				"6,3,Luke Skywalker\n" +
-				"8,4,Comment#2\n" +
-				"10,4,Comment#4\n" +
-				"12,5,Comment#6\n" +
-				"14,5,Comment#8\n" +
-				"16,6,Comment#10\n" +
-				"18,6,Comment#12\n" +
-				"20,6,Comment#14\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Filter4 implements FilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return (value.f0 % 2) == 0;
-		}
-	}
-
-	@Test
-	public void testFilterBasicType() throws Exception {
-		/*
-		 * Test filter on basic type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> filterDs = ds.
-				filter(new Filter5());
-		List<String> result = filterDs.collect();
-
-		String expected = "Hi\n" +
-				"Hello\n" +
-				"Hello world\n" +
-				"Hello world, how are you?\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Filter5 implements FilterFunction<String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(String value) throws Exception {
-			return value.startsWith("H");
-		}
-	}
-
-	@Test
-	public void testFilterOnCustomType() throws Exception {
-		/*
-		 * Test filter on custom type
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> filterDs = ds.
-				filter(new Filter6());
-		List<CustomType> result = filterDs.collect();
-
-		String expected = "3,3,Hello world, how are you?\n"
-				+
-				"3,4,I am fine.\n" +
-				"3,5,Luke Skywalker\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Filter6 implements FilterFunction<CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(CustomType value) throws Exception {
-			return value.myString.contains("a");
-		}
-	}
-
-	@Test
-	public void testRichFilterOnStringTupleField() throws Exception {
-		/*
-		 * Test filter on String tuple field.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new RichFilter1()).withBroadcastSet(ints, "ints");
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichFilter1 extends RichFilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		int literal = -1;
-
-		@Override
-		public void open(Configuration config) {
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			for(int i: ints) {
-				literal = literal < i ? i : literal;
-			}
-		}
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return value.f0 < literal;
-		}
-	}
-
-	@Test
-	public void testFilterWithBroadcastVariables() throws Exception {
-		/*
-		 * Test filter with broadcast variables
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
-				filter(new RichFilter2()).withBroadcastSet(intDs, "ints");
-		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
-
-		String expected = "11,5,Comment#5\n" +
-				"12,5,Comment#6\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" +
-				"15,5,Comment#9\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichFilter2 extends RichFilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-		private  int broadcastSum = 0;
-
-		@Override
-		public void open(Configuration config) {
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			for(Integer i : ints) {
-				broadcastSum += i;
-			}
-		}
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return (value.f1 == (broadcastSum / 11));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.java
deleted file mode 100644
index 3eb870d..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FirstNITCase.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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class FirstNITCase extends MultipleProgramsTestBase {
-	public FirstNITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testFirstNOnUngroupedDS() throws Exception {
-		/*
-		 * First-n on ungrouped data set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple1<Integer>> seven = ds.first(7).map(new OneMapper()).sum(0);
-
-		List<Tuple1<Integer>> result = seven.collect();
-
-		String expected = "(7)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testFirstNOnGroupedDS() throws Exception {
-		/*
-		 * First-n on grouped data set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Long, Integer>> first = ds.groupBy(1).first(4)
-				.map(new OneMapper2()).groupBy(0).sum(1);
-
-		List<Tuple2<Long, Integer>> result = first.collect();
-
-		String expected = "(1,1)\n(2,2)\n(3,3)\n(4,4)\n(5,4)\n(6,4)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testFirstNOnGroupedAndSortedDS() throws Exception {
-		/*
-		 * First-n on grouped and sorted data set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Long, Integer>> first = ds.groupBy(1).sortGroup(0, Order.DESCENDING).first(3)
-				.project(1,0);
-
-		List<Tuple2<Long, Integer>> result = first.collect();
-
-		String expected = "(1,1)\n"
-				+ "(2,3)\n(2,2)\n"
-				+ "(3,6)\n(3,5)\n(3,4)\n"
-				+ "(4,10)\n(4,9)\n(4,8)\n"
-				+ "(5,15)\n(5,14)\n(5,13)\n"
-				+ "(6,21)\n(6,20)\n(6,19)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	/**
-	 * Test for FLINK-2135
-	 */
-	@Test
-	public void testFaultyCast() throws Exception {
-		ExecutionEnvironment ee = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> b = ee.fromElements("a", "b");
-		GroupReduceOperator<String, String> a = b.groupBy(new KeySelector<String, Long>() {
-			@Override
-			public Long getKey(String value) throws Exception {
-				return 1L;
-			}
-		}).sortGroup(new KeySelector<String, Double>() {
-			@Override
-			public Double getKey(String value) throws Exception {
-				return 1.0;
-			}
-		}, Order.DESCENDING).first(1);
-
-		List<String> result = b.collect();
-
-		String expected = "a\nb";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class OneMapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple1<Integer>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple1<Integer> one = new Tuple1<Integer>(1);
-		@Override
-		public Tuple1<Integer> map(Tuple3<Integer, Long, String> value) {
-			return one;
-		}
-	}
-
-	public static class OneMapper2 implements MapFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple2<Long, Integer> one = new Tuple2<Long, Integer>(0l,1);
-		@Override
-		public Tuple2<Long, Integer> map(Tuple3<Integer, Long, String> value) {
-			one.f0 = value.f1;
-			return one;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
deleted file mode 100644
index 4962da8..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/FlatMapITCase.java
+++ /dev/null
@@ -1,360 +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.javaApiOperators;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class FlatMapITCase extends MultipleProgramsTestBase {
-	public FlatMapITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testNonPassingFlatMap() throws Exception {
-		/*
-		 * Test non-passing flatmap
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> nonPassingFlatMapDs = ds.
-				flatMap(new FlatMapper1());
-
-		List<String> result = nonPassingFlatMapDs.collect();
-
-		String expected = "\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class FlatMapper1 implements FlatMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<String> out) throws Exception {
-			if ( value.contains("bananas") ) {
-				out.collect(value);
-			}
-		}
-	}
-
-	@Test
-	public void testDataDuplicatingFlatMap() throws Exception {
-		/*
-		 * Test data duplicating flatmap
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> duplicatingFlatMapDs = ds.
-				flatMap(new FlatMapper2());
-
-		List<String> result = duplicatingFlatMapDs.collect();
-
-		String expected = "Hi\n" + "HI\n" +
-				"Hello\n" + "HELLO\n" +
-				"Hello world\n" + "HELLO WORLD\n" +
-				"Hello world, how are you?\n" + "HELLO WORLD, HOW ARE YOU?\n" +
-				"I am fine.\n" + "I AM FINE.\n" +
-				"Luke Skywalker\n" + "LUKE SKYWALKER\n" +
-				"Random comment\n" + "RANDOM COMMENT\n" +
-				"LOL\n" + "LOL\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class FlatMapper2 implements FlatMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<String> out) throws Exception {
-			out.collect(value);
-			out.collect(value.toUpperCase());
-		}
-	}
-
-	@Test
-	public void testFlatMapWithVaryingNumberOfEmittedTuples() throws Exception {
-		/*
-		 * Test flatmap with varying number of emitted tuples
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> varyingTuplesMapDs = ds.
-				flatMap(new FlatMapper3());
-
-		List<Tuple3<Integer, Long, String>> result = varyingTuplesMapDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" + "2,2,Hello\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n" + "5,3,I am fine.\n" +
-				"7,4,Comment#1\n" +
-				"8,4,Comment#2\n" + "8,4,Comment#2\n" +
-				"10,4,Comment#4\n" +
-				"11,5,Comment#5\n" + "11,5,Comment#5\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" + "14,5,Comment#8\n" +
-				"16,6,Comment#10\n" +
-				"17,6,Comment#11\n" + "17,6,Comment#11\n" +
-				"19,6,Comment#13\n" +
-				"20,6,Comment#14\n" + "20,6,Comment#14\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class FlatMapper3 implements FlatMapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Tuple3<Integer, Long, String> value,
-				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			final int numTuples = value.f0 % 3;
-			for ( int i = 0; i < numTuples; i++ ) {
-				out.collect(value);
-			}
-		}
-	}
-
-	@Test
-	public void testTypeConversionFlatMapperCustomToTuple() throws Exception {
-		/*
-		 * Test type conversion flatmapper (Custom -> Tuple)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> typeConversionFlatMapDs = ds.
-				flatMap(new FlatMapper4());
-
-		List<Tuple3<Integer, Long, String>> result = typeConversionFlatMapDs.collect();
-
-		String expected = "1,0,Hi\n" +
-				"2,1,Hello\n" +
-				"2,2,Hello world\n" +
-				"3,3,Hello world, how are you?\n" +
-				"3,4,I am fine.\n" +
-				"3,5,Luke Skywalker\n" +
-				"4,6,Comment#1\n" +
-				"4,7,Comment#2\n" +
-				"4,8,Comment#3\n" +
-				"4,9,Comment#4\n" +
-				"5,10,Comment#5\n" +
-				"5,11,Comment#6\n" +
-				"5,12,Comment#7\n" +
-				"5,13,Comment#8\n" +
-				"5,14,Comment#9\n" +
-				"6,15,Comment#10\n" +
-				"6,16,Comment#11\n" +
-				"6,17,Comment#12\n" +
-				"6,18,Comment#13\n" +
-				"6,19,Comment#14\n" +
-				"6,20,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class FlatMapper4 implements FlatMapFunction<CustomType, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> outTuple =
-				new Tuple3<Integer, Long, String>();
-
-		@Override
-		public void flatMap(CustomType value, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			outTuple.setField(value.myInt, 0);
-			outTuple.setField(value.myLong, 1);
-			outTuple.setField(value.myString, 2);
-			out.collect(outTuple);
-		}
-	}
-
-	@Test
-	public void testTypeConversionFlatMapperTupleToBasic() throws Exception {
-		/*
-		 * Test type conversion flatmapper (Tuple -> Basic)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<String> typeConversionFlatMapDs = ds.
-				flatMap(new FlatMapper5());
-
-		List<String> result = typeConversionFlatMapDs.collect();
-
-		String expected = "Hi\n" + "Hello\n" + "Hello world\n"
-				+
-				"Hello world, how are you?\n" +
-				"I am fine.\n" + "Luke Skywalker\n" +
-				"Comment#1\n" +	"Comment#2\n" +
-				"Comment#3\n" +	"Comment#4\n" +
-				"Comment#5\n" +	"Comment#6\n" +
-				"Comment#7\n" + "Comment#8\n" +
-				"Comment#9\n" +	"Comment#10\n" +
-				"Comment#11\n" + "Comment#12\n" +
-				"Comment#13\n" + "Comment#14\n" +
-				"Comment#15\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class FlatMapper5 implements FlatMapFunction<Tuple3<Integer, Long, String>,String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Tuple3<Integer, Long, String> value, Collector<String> out) throws Exception {
-			out.collect(value.f2);
-		}
-	}
-
-	@Test
-	public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() throws Exception {
-		/*
-		 * Test flatmapper if UDF returns input object
-		 * multiple times and changes it in between
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> inputObjFlatMapDs = ds.
-				flatMap(new FlatMapper6());
-
-		List<Tuple3<Integer, Long, String>> result = inputObjFlatMapDs.collect();
-
-		String expected = "0,1,Hi\n" +
-				"0,2,Hello\n" + "1,2,Hello\n" +
-				"0,2,Hello world\n" + "1,2,Hello world\n" + "2,2,Hello world\n" +
-				"0,3,I am fine.\n" +
-				"0,3,Luke Skywalker\n" + "1,3,Luke Skywalker\n" +
-				"0,4,Comment#1\n" + "1,4,Comment#1\n" + "2,4,Comment#1\n" +
-				"0,4,Comment#3\n" +
-				"0,4,Comment#4\n" + "1,4,Comment#4\n" +
-				"0,5,Comment#5\n" + "1,5,Comment#5\n" + "2,5,Comment#5\n" +
-				"0,5,Comment#7\n" +
-				"0,5,Comment#8\n" + "1,5,Comment#8\n" +
-				"0,5,Comment#9\n" + "1,5,Comment#9\n" + "2,5,Comment#9\n" +
-				"0,6,Comment#11\n" +
-				"0,6,Comment#12\n" + "1,6,Comment#12\n" +
-				"0,6,Comment#13\n" + "1,6,Comment#13\n" + "2,6,Comment#13\n" +
-				"0,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class FlatMapper6 implements FlatMapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap( Tuple3<Integer, Long, String> value,
-				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			final int numTuples = value.f0 % 4;
-			for ( int i = 0; i < numTuples; i++ ) {
-				value.setField(i, 0);
-				out.collect(value);
-			}
-		}
-	}
-
-	@Test
-	public void testFlatMapWithBroadcastSet() throws Exception {
-		/*
-		 * Test flatmap with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> bcFlatMapDs = ds.
-				flatMap(new RichFlatMapper1()).withBroadcastSet(ints, "ints");
-		List<Tuple3<Integer, Long, String>> result = bcFlatMapDs.collect();
-
-		String expected = "55,1,Hi\n" +
-				"55,2,Hello\n" +
-				"55,2,Hello world\n" +
-				"55,3,Hello world, how are you?\n" +
-				"55,3,I am fine.\n" +
-				"55,3,Luke Skywalker\n" +
-				"55,4,Comment#1\n" +
-				"55,4,Comment#2\n" +
-				"55,4,Comment#3\n" +
-				"55,4,Comment#4\n" +
-				"55,5,Comment#5\n" +
-				"55,5,Comment#6\n" +
-				"55,5,Comment#7\n" +
-				"55,5,Comment#8\n" +
-				"55,5,Comment#9\n" +
-				"55,6,Comment#10\n" +
-				"55,6,Comment#11\n" +
-				"55,6,Comment#12\n" +
-				"55,6,Comment#13\n" +
-				"55,6,Comment#14\n" +
-				"55,6,Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichFlatMapper1 extends RichFlatMapFunction<Tuple3<Integer,Long,String>, Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> outTuple =
-				new Tuple3<Integer, Long, String>();
-		private Integer f2Replace = 0;
-
-		@Override
-		public void open(Configuration config) {
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			f2Replace = sum;
-		}
-
-		@Override
-		public void flatMap(Tuple3<Integer, Long, String> value,
-				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			outTuple.setFields(f2Replace, value.f1, value.f2);
-			out.collect(outTuple);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
deleted file mode 100644
index 9b56c63..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupCombineITCase.java
+++ /dev/null
@@ -1,482 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.operators.UnsortedGrouping;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.List;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-/**
- * The GroupCombine operator is not easy to test because it is essentially just a combiner. The result can be
- * the result of a normal groupReduce at any stage its execution. The basic idea is to preserve the grouping key
- * in the partial result, so that we can do a reduceGroup afterwards to finalize the results for verification.
- * In addition, we can use hashPartition to partition the data and check if no shuffling (just combining) has
- * been performed.
- */
-public class GroupCombineITCase extends MultipleProgramsTestBase {
-
-	public GroupCombineITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private static String identityResult = "1,1,Hi\n" +
-			"2,2,Hello\n" +
-			"3,2,Hello world\n" +
-			"4,3,Hello world, how are you?\n" +
-			"5,3,I am fine.\n" +
-			"6,3,Luke Skywalker\n" +
-			"7,4,Comment#1\n" +
-			"8,4,Comment#2\n" +
-			"9,4,Comment#3\n" +
-			"10,4,Comment#4\n" +
-			"11,5,Comment#5\n" +
-			"12,5,Comment#6\n" +
-			"13,5,Comment#7\n" +
-			"14,5,Comment#8\n" +
-			"15,5,Comment#9\n" +
-			"16,6,Comment#10\n" +
-			"17,6,Comment#11\n" +
-			"18,6,Comment#12\n" +
-			"19,6,Comment#13\n" +
-			"20,6,Comment#14\n" +
-			"21,6,Comment#15\n";
-
-	@Test
-	public void testAllGroupCombineIdentity() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
-				// combine
-				.combineGroup(new IdentityFunction())
-				// fully reduce
-				.reduceGroup(new IdentityFunction());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		compareResultAsTuples(result, identityResult);
-	}
-
-	@Test
-	public void testIdentity() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
-				// combine
-				.combineGroup(new IdentityFunction())
-				// fully reduce
-				.reduceGroup(new IdentityFunction());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		compareResultAsTuples(result, identityResult);
-	}
-
-	@Test
-	public void testIdentityWithGroupBy() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
-				.groupBy(1)
-				// combine
-				.combineGroup(new IdentityFunction())
-				// fully reduce
-				.reduceGroup(new IdentityFunction());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		compareResultAsTuples(result, identityResult);
-	}
-
-	@Test
-	public void testIdentityWithGroupByAndSort() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
-				.groupBy(1)
-				.sortGroup(1, Order.DESCENDING)
-				// reduce partially
-				.combineGroup(new IdentityFunction())
-				.groupBy(1)
-				.sortGroup(1, Order.DESCENDING)
-				// fully reduce
-				.reduceGroup(new IdentityFunction());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		compareResultAsTuples(result, identityResult);
-	}
-
-	@Test
-	public void testPartialReduceWithIdenticalInputOutputType() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// data
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple2<Long, Tuple3<Integer, Long, String>>> dsWrapped = ds
-				// wrap values as Kv pairs with the grouping key as key
-				.map(new Tuple3KvWrapper());
-
-		List<Tuple3<Integer, Long, String>> result = dsWrapped
-				.groupBy(0)
-				// reduce partially
-				.combineGroup(new Tuple3toTuple3GroupReduce())
-				.groupBy(0)
-				// reduce fully to check result
-				.reduceGroup(new Tuple3toTuple3GroupReduce())
-				//unwrap
-				.map(new MapFunction<Tuple2<Long, Tuple3<Integer, Long, String>>, Tuple3<Integer, Long, String>>() {
-					@Override
-					public Tuple3<Integer, Long, String> map(Tuple2<Long, Tuple3<Integer, Long, String>> value) throws Exception {
-						return value.f1;
-					}
-				}).collect();
-
-		String expected = "1,1,combined\n" +
-				"5,4,combined\n" +
-				"15,9,combined\n" +
-				"34,16,combined\n" +
-				"65,25,combined\n" +
-				"111,36,combined\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testPartialReduceWithDifferentInputOutputType() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// data
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple2<Long, Tuple3<Integer, Long, String>>> dsWrapped = ds
-				// wrap values as Kv pairs with the grouping key as key
-				.map(new Tuple3KvWrapper());
-
-		List<Tuple2<Integer, Long>> result = dsWrapped
-				.groupBy(0)
-				// reduce partially
-				.combineGroup(new Tuple3toTuple2GroupReduce())
-				.groupBy(0)
-				// reduce fully to check result
-				.reduceGroup(new Tuple2toTuple2GroupReduce())
-				//unwrap
-				.map(new MapFunction<Tuple2<Long,Tuple2<Integer,Long>>, Tuple2<Integer,Long>>() {
-					@Override
-					public Tuple2<Integer, Long> map(Tuple2<Long, Tuple2<Integer, Long>> value) throws Exception {
-						return value.f1;
-					}
-				}).collect();
-
-		String expected = "1,3\n" +
-				"5,20\n" +
-				"15,58\n" +
-				"34,52\n" +
-				"65,70\n" +
-				"111,96\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	// check if no shuffle is being executed
-	public void testCheckPartitionShuffleGroupBy() throws Exception {
-
-		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// data
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		// partition and group data
-		UnsortedGrouping<Tuple3<Integer, Long, String>> partitionedDS = ds.partitionByHash(0).groupBy(1);
-
-		List<Tuple2<Long, Integer>> result = partitionedDS
-				.combineGroup(
-						new GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>>() {
-			@Override
-			public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Long, Integer>> out) throws Exception {
-				int count = 0;
-				long key = 0;
-				for (Tuple3<Integer, Long, String> value : values) {
-					key = value.f1;
-					count++;
-				}
-				out.collect(new Tuple2<>(key, count));
-			}
-		}).collect();
-
-		String[] localExpected = new String[] { "(6,6)", "(5,5)" + "(4,4)", "(3,3)", "(2,2)", "(1,1)" };
-
-		String[] resultAsStringArray = new String[result.size()];
-		for (int i = 0; i < resultAsStringArray.length; ++i) {
-			resultAsStringArray[i] = result.get(i).toString();
-		}
-		Arrays.sort(resultAsStringArray);
-
-		Assert.assertEquals("The two arrays were identical.", false, Arrays.equals(localExpected, resultAsStringArray));
-	}
-
-	@Test
-	// check if parallelism of 1 results in the same data like a shuffle
-	public void testCheckPartitionShuffleDOP1() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		env.setParallelism(1);
-
-		// data
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		// partition and group data
-		UnsortedGrouping<Tuple3<Integer, Long, String>> partitionedDS = ds.partitionByHash(0).groupBy(1);
-
-		List<Tuple2<Long, Integer>> result = partitionedDS
-				.combineGroup(
-				new GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>>() {
-					@Override
-					public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Long, Integer>> out) throws Exception {
-						int count = 0;
-						long key = 0;
-						for (Tuple3<Integer, Long, String> value : values) {
-							key = value.f1;
-							count++;
-						}
-						out.collect(new Tuple2<>(key, count));
-					}
-				}).collect();
-
-		String expected = "6,6\n" +
-				"5,5\n" +
-				"4,4\n" +
-				"3,3\n" +
-				"2,2\n" +
-				"1,1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	// check if all API methods are callable
-	public void testAPI() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple1<String>> ds = CollectionDataSets.getStringDataSet(env).map(new MapFunction<String, Tuple1<String>>() {
-			@Override
-			public Tuple1<String> map(String value) throws Exception {
-				return new Tuple1<>(value);
-			}
-		});
-
-		// all methods on DataSet
-		ds.combineGroup(new GroupCombineFunctionExample())
-		.output(new DiscardingOutputFormat<Tuple1<String>>());
-
-		// all methods on UnsortedGrouping
-		ds.groupBy(0).combineGroup(new GroupCombineFunctionExample())
-		.output(new DiscardingOutputFormat<Tuple1<String>>());
-
-		// all methods on SortedGrouping
-		ds.groupBy(0).sortGroup(0, Order.ASCENDING).combineGroup(new GroupCombineFunctionExample())
-		.output(new DiscardingOutputFormat<Tuple1<String>>());
-
-		env.execute();
-	}
-
-	public static class GroupCombineFunctionExample implements GroupCombineFunction<Tuple1<String>, Tuple1<String>> {
-
-		@Override
-		public void combine(Iterable<Tuple1<String>> values, Collector<Tuple1<String>> out) throws Exception {
-			for (Tuple1<String> value : values) {
-				out.collect(value);
-			}
-		}
-	}
-
-	public static class ScalaGroupCombineFunctionExample implements GroupCombineFunction<scala.Tuple1<String>, scala.Tuple1<String>> {
-
-		@Override
-		public void combine(Iterable<scala.Tuple1<String>> values, Collector<scala.Tuple1<String>> out) throws Exception {
-			for (scala.Tuple1<String> value : values) {
-				out.collect(value);
-			}
-		}
-	}
-
-	public static class IdentityFunction implements GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>,
-	GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-
-		@Override
-		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			for (Tuple3<Integer, Long, String> value : values) {
-				out.collect(new Tuple3<>(value.f0, value.f1, value.f2));
-			}
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			for (Tuple3<Integer, Long, String> value : values) {
-				out.collect(new Tuple3<>(value.f0, value.f1, value.f2));
-			}
-		}
-	}
-
-
-	public static class Tuple3toTuple3GroupReduce implements KvGroupReduce<Long, Tuple3<Integer, Long, String>,
-			Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-
-		@Override
-		public void combine(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values, Collector<Tuple2<Long,
-				Tuple3<Integer, Long, String>>> out) throws Exception {
-			int i = 0;
-			long l = 0;
-			long key = 0;
-
-			// collapse groups
-			for (Tuple2<Long, Tuple3<Integer, Long, String>> value : values) {
-				key = value.f0;
-				Tuple3<Integer, Long, String> extracted = value.f1;
-				i += extracted.f0;
-				l += extracted.f1;
-			}
-
-			Tuple3<Integer, Long, String> result = new Tuple3<>(i, l, "combined");
-			out.collect(new Tuple2<>(key, result));
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values,
-											 Collector<Tuple2<Long, Tuple3<Integer, Long, String>>> out) throws Exception {
-			combine(values, out);
-		}
-	}
-
-	public static class Tuple3toTuple2GroupReduce implements KvGroupReduce<Long, Tuple3<Integer, Long, String>,
-			Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
-
-		@Override
-		public void combine(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values, Collector<Tuple2<Long,
-				Tuple2<Integer, Long>>> out) throws Exception {
-			int i = 0;
-			long l = 0;
-			long key = 0;
-
-			// collapse groups
-			for (Tuple2<Long, Tuple3<Integer, Long, String>> value : values) {
-				key = value.f0;
-				Tuple3<Integer, Long, String> extracted = value.f1;
-				i += extracted.f0;
-				l += extracted.f1 + extracted.f2.length();
-			}
-
-			Tuple2<Integer, Long> result = new Tuple2<>(i, l);
-			out.collect(new Tuple2<>(key, result));
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long,
-				Tuple2<Integer, Long>>> out) throws Exception {
-			new Tuple2toTuple2GroupReduce().reduce(values, out);
-		}
-	}
-
-	public static class Tuple2toTuple2GroupReduce implements KvGroupReduce<Long, Tuple2<Integer, Long>,
-			Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
-
-		@Override
-		public void combine(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long, Tuple2<Integer,
-				Long>>> out) throws Exception {
-			int i = 0;
-			long l = 0;
-			long key = 0;
-
-			// collapse groups
-			for (Tuple2<Long, Tuple2<Integer, Long>> value : values) {
-				key = value.f0;
-				Tuple2<Integer, Long> extracted = value.f1;
-				i += extracted.f0;
-				l += extracted.f1;
-			}
-
-			Tuple2<Integer, Long> result = new Tuple2<>(i, l);
-
-			out.collect(new Tuple2<>(key, result));
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long,
-				Tuple2<Integer, Long>>> out) throws Exception {
-			combine(values, out);
-		}
-	}
-
-	public class Tuple3KvWrapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple2<Long,
-			Tuple3<Integer, Long, String>>> {
-		@Override
-		public Tuple2<Long, Tuple3<Integer, Long, String>> map(Tuple3<Integer, Long, String> value) throws Exception {
-			return new Tuple2<>(value.f1, value);
-		}
-	}
-
-
-	public interface CombineAndReduceGroup <IN, INT, OUT> extends GroupCombineFunction<IN, INT>,
-			GroupReduceFunction<INT, OUT> {
-	}
-
-	public interface KvGroupReduce<K, V, INT, OUT> extends CombineAndReduceGroup<Tuple2<K, V>, Tuple2<K, INT>,
-			Tuple2<K, OUT>> {
-	}
-
-}


[11/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java
deleted file mode 100644
index ba48e12..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/CollectionDataSets.java
+++ /dev/null
@@ -1,725 +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.javaApiOperators.util;
-
-import java.io.File;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.hadoop.io.IntWritable;
-
-import scala.math.BigInt;
-
-/**
- * #######################################################################################################
- * 
- * 			BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. 
- * 			IF YOU MODIFY THE DATA MAKE SURE YOU CHECK THAT ALL TESTS ARE STILL WORKING!
- * 
- * #######################################################################################################
- */
-public class CollectionDataSets {
-
-	public static DataSet<Tuple3<Integer, Long, String>> get3TupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
-		data.add(new Tuple3<>(1, 1L, "Hi"));
-		data.add(new Tuple3<>(2, 2L, "Hello"));
-		data.add(new Tuple3<>(3, 2L, "Hello world"));
-		data.add(new Tuple3<>(4, 3L, "Hello world, how are you?"));
-		data.add(new Tuple3<>(5, 3L, "I am fine."));
-		data.add(new Tuple3<>(6, 3L, "Luke Skywalker"));
-		data.add(new Tuple3<>(7, 4L, "Comment#1"));
-		data.add(new Tuple3<>(8, 4L, "Comment#2"));
-		data.add(new Tuple3<>(9, 4L, "Comment#3"));
-		data.add(new Tuple3<>(10, 4L, "Comment#4"));
-		data.add(new Tuple3<>(11, 5L, "Comment#5"));
-		data.add(new Tuple3<>(12, 5L, "Comment#6"));
-		data.add(new Tuple3<>(13, 5L, "Comment#7"));
-		data.add(new Tuple3<>(14, 5L, "Comment#8"));
-		data.add(new Tuple3<>(15, 5L, "Comment#9"));
-		data.add(new Tuple3<>(16, 6L, "Comment#10"));
-		data.add(new Tuple3<>(17, 6L, "Comment#11"));
-		data.add(new Tuple3<>(18, 6L, "Comment#12"));
-		data.add(new Tuple3<>(19, 6L, "Comment#13"));
-		data.add(new Tuple3<>(20, 6L, "Comment#14"));
-		data.add(new Tuple3<>(21, 6L, "Comment#15"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple3<Integer, Long, String>> getSmall3TupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
-		data.add(new Tuple3<>(1, 1L, "Hi"));
-		data.add(new Tuple3<>(2, 2L, "Hello"));
-		data.add(new Tuple3<>(3, 2L, "Hello world"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple5<Integer, Long, Integer, String, Long>> get5TupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
-		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
-		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
-		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
-		data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L));
-		data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L));
-		data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L));
-		data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L));
-		data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L));
-		data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L));
-		data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L));
-		data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L));
-		data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L));
-		data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L));
-		data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L));
-		data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L));
-
-		Collections.shuffle(data);
-
-		TupleTypeInfo<Tuple5<Integer, Long, Integer, String, Long>> type = new TupleTypeInfo<>(
-				BasicTypeInfo.INT_TYPE_INFO,
-				BasicTypeInfo.LONG_TYPE_INFO,
-				BasicTypeInfo.INT_TYPE_INFO,
-				BasicTypeInfo.STRING_TYPE_INFO,
-				BasicTypeInfo.LONG_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple5<Integer, Long, Integer, String, Long>> getSmall5TupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
-		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
-		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
-		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
-
-		Collections.shuffle(data);
-
-		TupleTypeInfo<Tuple5<Integer, Long, Integer, String, Long>> type = new TupleTypeInfo<>(
-				BasicTypeInfo.INT_TYPE_INFO,
-				BasicTypeInfo.LONG_TYPE_INFO,
-				BasicTypeInfo.INT_TYPE_INFO,
-				BasicTypeInfo.STRING_TYPE_INFO,
-				BasicTypeInfo.LONG_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple2<Tuple2<Integer, Integer>, String>> getSmallNestedTupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple2<Tuple2<Integer, Integer>, String>> data = new ArrayList<>();
-		data.add(new Tuple2<>(new Tuple2<>(1, 1), "one"));
-		data.add(new Tuple2<>(new Tuple2<>(2, 2), "two"));
-		data.add(new Tuple2<>(new Tuple2<>(3, 3), "three"));
-
-		TupleTypeInfo<Tuple2<Tuple2<Integer, Integer>, String>> type = new TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
-				BasicTypeInfo.STRING_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple2<Tuple2<Integer, Integer>, String>> getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) {
-
-		List<Tuple2<Tuple2<Integer, Integer>, String>> data = new ArrayList<>();
-		data.add(new Tuple2<>(new Tuple2<>(1, 3), "a"));
-		data.add(new Tuple2<>(new Tuple2<>(1, 2), "a"));
-		data.add(new Tuple2<>(new Tuple2<>(2, 1), "a"));
-		data.add(new Tuple2<>(new Tuple2<>(2, 2), "b"));
-		data.add(new Tuple2<>(new Tuple2<>(3, 3), "c"));
-		data.add(new Tuple2<>(new Tuple2<>(3, 6), "c"));
-		data.add(new Tuple2<>(new Tuple2<>(4, 9), "c"));
-
-		TupleTypeInfo<Tuple2<Tuple2<Integer, Integer>, String>> type = new TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
-				BasicTypeInfo.STRING_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) {
-
-		List<Tuple3<Tuple2<Integer, Integer>, String, Integer>> data = new ArrayList<>();
-		data.add(new Tuple3<>(new Tuple2<>(1, 3), "a", 2));
-		data.add(new Tuple3<>(new Tuple2<>(1, 2), "a", 1));
-		data.add(new Tuple3<>(new Tuple2<>(2, 1), "a", 3));
-		data.add(new Tuple3<>(new Tuple2<>(2, 2), "b", 4));
-		data.add(new Tuple3<>(new Tuple2<>(3, 3), "c", 5));
-		data.add(new Tuple3<>(new Tuple2<>(3, 6), "c", 6));
-		data.add(new Tuple3<>(new Tuple2<>(4, 9), "c", 7));
-
-		TupleTypeInfo<Tuple3<Tuple2<Integer, Integer>, String, Integer>> type = new TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO),
-				BasicTypeInfo.STRING_TYPE_INFO,
-				BasicTypeInfo.INT_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-	
-	public static DataSet<Tuple2<byte[], Integer>> getTuple2WithByteArrayDataSet(ExecutionEnvironment env) {
-		List<Tuple2<byte[], Integer>> data = new ArrayList<>();
-		data.add(new Tuple2<>(new byte[]{0, 4}, 1));
-		data.add(new Tuple2<>(new byte[]{2, 0}, 1));
-		data.add(new Tuple2<>(new byte[]{2, 0, 4}, 4));
-		data.add(new Tuple2<>(new byte[]{2, 1}, 3));
-		data.add(new Tuple2<>(new byte[]{0}, 0));
-		data.add(new Tuple2<>(new byte[]{2, 0}, 1));
-				
-		TupleTypeInfo<Tuple2<byte[], Integer>> type = new TupleTypeInfo<>(
-				PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO,
-				BasicTypeInfo.INT_TYPE_INFO
-		);
-		
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<String> getStringDataSet(ExecutionEnvironment env) {
-
-		List<String> data = new ArrayList<>();
-		data.add("Hi");
-		data.add("Hello");
-		data.add("Hello world");
-		data.add("Hello world, how are you?");
-		data.add("I am fine.");
-		data.add("Luke Skywalker");
-		data.add("Random comment");
-		data.add("LOL");
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Integer> getIntegerDataSet(ExecutionEnvironment env) {
-
-		List<Integer> data = new ArrayList<>();
-		data.add(1);
-		data.add(2);
-		data.add(2);
-		data.add(3);
-		data.add(3);
-		data.add(3);
-		data.add(4);
-		data.add(4);
-		data.add(4);
-		data.add(4);
-		data.add(5);
-		data.add(5);
-		data.add(5);
-		data.add(5);
-		data.add(5);
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<CustomType> getCustomTypeDataSet(ExecutionEnvironment env) {
-
-		List<CustomType> data = new ArrayList<>();
-		data.add(new CustomType(1, 0L, "Hi"));
-		data.add(new CustomType(2, 1L, "Hello"));
-		data.add(new CustomType(2, 2L, "Hello world"));
-		data.add(new CustomType(3, 3L, "Hello world, how are you?"));
-		data.add(new CustomType(3, 4L, "I am fine."));
-		data.add(new CustomType(3, 5L, "Luke Skywalker"));
-		data.add(new CustomType(4, 6L, "Comment#1"));
-		data.add(new CustomType(4, 7L, "Comment#2"));
-		data.add(new CustomType(4, 8L, "Comment#3"));
-		data.add(new CustomType(4, 9L, "Comment#4"));
-		data.add(new CustomType(5, 10L, "Comment#5"));
-		data.add(new CustomType(5, 11L, "Comment#6"));
-		data.add(new CustomType(5, 12L, "Comment#7"));
-		data.add(new CustomType(5, 13L, "Comment#8"));
-		data.add(new CustomType(5, 14L, "Comment#9"));
-		data.add(new CustomType(6, 15L, "Comment#10"));
-		data.add(new CustomType(6, 16L, "Comment#11"));
-		data.add(new CustomType(6, 17L, "Comment#12"));
-		data.add(new CustomType(6, 18L, "Comment#13"));
-		data.add(new CustomType(6, 19L, "Comment#14"));
-		data.add(new CustomType(6, 20L, "Comment#15"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-
-	}
-
-	public static DataSet<CustomType> getSmallCustomTypeDataSet(ExecutionEnvironment env) {
-
-		List<CustomType> data = new ArrayList<>();
-		data.add(new CustomType(1, 0L, "Hi"));
-		data.add(new CustomType(2, 1L, "Hello"));
-		data.add(new CustomType(2, 2L, "Hello world"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-
-	}
-
-	public static class CustomType implements Serializable {
-
-		private static final long serialVersionUID = 1L;
-
-		public int myInt;
-		public long myLong;
-		public String myString;
-
-		public CustomType() {
-		}
-
-		public CustomType(int i, long l, String s) {
-			myInt = i;
-			myLong = l;
-			myString = s;
-		}
-
-		@Override
-		public String toString() {
-			return myInt + "," + myLong + "," + myString;
-		}
-	}
-
-	public static class CustomTypeComparator implements Comparator<CustomType> {
-		@Override
-		public int compare(CustomType o1, CustomType o2) {
-			int diff = o1.myInt - o2.myInt;
-			if (diff != 0) {
-				return diff;
-			}
-			diff = (int) (o1.myLong - o2.myLong);
-			return diff != 0 ? diff : o1.myString.compareTo(o2.myString);
-		}
-
-	}
-
-	public static DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> getSmallTuplebasedDataSet(ExecutionEnvironment env) {
-		List<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> data = new ArrayList<>();
-		data.add(new Tuple7<>(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new Tuple7<>(2, "Second", 20, 200, 2000L, "Two", 20000L));
-		data.add(new Tuple7<>(3, "Third", 30, 300, 3000L, "Three", 30000L));
-		return env.fromCollection(data);
-	}
-	
-	public static DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) {
-		List<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> data = new ArrayList<>();
-		data.add(new Tuple7<>(10000L, 10, 100, 1000L, "One", 1, "First"));
-		data.add(new Tuple7<>(20000L, 20, 200, 2000L, "Two", 2, "Second"));
-		data.add(new Tuple7<>(30000L, 30, 300, 3000L, "Three", 3, "Third"));
-		
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getSmallPojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<>();
-		data.add(new POJO(1 /*number*/, "First" /*str*/, 10 /*f0*/, 100/*f1.myInt*/, 1000L/*f1.myLong*/, "One" /*f1.myString*/, 10000L /*nestedPojo.longNumber*/));
-		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getDuplicatePojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<>();
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getMixedPojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<>();
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x
-		data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L));
-		data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L));
-		data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L));
-		data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L));
-		data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L));
-		data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x
-		data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L));
-		return env.fromCollection(data);
-	}
-
-	public static class POJO {
-		public int number;
-		public String str;
-		public Tuple2<Integer, CustomType> nestedTupleWithCustom;
-		public NestedPojo nestedPojo;
-		public transient Long ignoreMe;
-
-		public POJO(int i0, String s0,
-					int i1, int i2, long l0, String s1,
-					long l1) {
-			this.number = i0;
-			this.str = s0;
-			this.nestedTupleWithCustom = new Tuple2<>(i1, new CustomType(i2, l0, s1));
-			this.nestedPojo = new NestedPojo();
-			this.nestedPojo.longNumber = l1;
-		}
-
-		public POJO() {
-		}
-
-		@Override
-		public String toString() {
-			return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber;
-		}
-	}
-
-	public static class NestedPojo {
-		public static Object ignoreMe;
-		public long longNumber;
-
-		public NestedPojo() {
-		}
-	}
-
-	public static DataSet<CrazyNested> getCrazyNestedDataSet(ExecutionEnvironment env) {
-		List<CrazyNested> data = new ArrayList<>();
-		data.add(new CrazyNested("aa"));
-		data.add(new CrazyNested("bb"));
-		data.add(new CrazyNested("bb"));
-		data.add(new CrazyNested("cc"));
-		data.add(new CrazyNested("cc"));
-		data.add(new CrazyNested("cc"));
-		return env.fromCollection(data);
-	}
-
-	public static class CrazyNested {
-		public CrazyNestedL1 nest_Lvl1;
-		public Long something; // test proper null-value handling
-
-		public CrazyNested() {
-		}
-
-		public CrazyNested(String set, String second, long s) { // additional CTor to set all fields to non-null values
-			this(set);
-			something = s;
-			nest_Lvl1.a = second;
-		}
-
-		public CrazyNested(String set) {
-			nest_Lvl1 = new CrazyNestedL1();
-			nest_Lvl1.nest_Lvl2 = new CrazyNestedL2();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3 = new CrazyNestedL3();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4 = new CrazyNestedL4();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal = set;
-		}
-	}
-
-	public static class CrazyNestedL1 {
-		public String a;
-		public int b;
-		public CrazyNestedL2 nest_Lvl2;
-	}
-
-	public static class CrazyNestedL2 {
-		public CrazyNestedL3 nest_Lvl3;
-	}
-
-	public static class CrazyNestedL3 {
-		public CrazyNestedL4 nest_Lvl4;
-	}
-
-	public static class CrazyNestedL4 {
-		public String f1nal;
-	}
-
-	// Copied from TypeExtractorTest
-	public static class FromTuple extends Tuple3<String, String, Long> {
-		private static final long serialVersionUID = 1L;
-		public int special;
-	}
-
-	public static class FromTupleWithCTor extends FromTuple {
-
-		private static final long serialVersionUID = 1L;
-
-		public FromTupleWithCTor() {}
-
-		public FromTupleWithCTor(int special, long tupleField) {
-			this.special = special;
-			this.setField(tupleField, 2);
-		}
-	}
-
-	public static DataSet<FromTupleWithCTor> getPojoExtendingFromTuple(ExecutionEnvironment env) {
-		List<FromTupleWithCTor> data = new ArrayList<>();
-		data.add(new FromTupleWithCTor(1, 10L)); // 3x
-		data.add(new FromTupleWithCTor(1, 10L));
-		data.add(new FromTupleWithCTor(1, 10L));
-		data.add(new FromTupleWithCTor(2, 20L)); // 2x
-		data.add(new FromTupleWithCTor(2, 20L));
-		return env.fromCollection(data);
-	}
-
-	public static class PojoContainingTupleAndWritable {
-		public int someInt;
-		public String someString;
-		public IntWritable hadoopFan;
-		public Tuple2<Long, Long> theTuple;
-
-		public PojoContainingTupleAndWritable() {
-		}
-
-		public PojoContainingTupleAndWritable(int i, long l1, long l2) {
-			hadoopFan = new IntWritable(i);
-			someInt = i;
-			theTuple = new Tuple2<>(l1, l2);
-		}
-	}
-
-	public static DataSet<PojoContainingTupleAndWritable> getPojoContainingTupleAndWritable(ExecutionEnvironment env) {
-		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
-		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		return env.fromCollection(data);
-	}
-
-
-
-	public static DataSet<PojoContainingTupleAndWritable> getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) {
-		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
-		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 201L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 600L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 400L));
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple3<Integer, CrazyNested, POJO>> getTupleContainingPojos(ExecutionEnvironment env) {
-		List<Tuple3<Integer, CrazyNested, POJO>> data = new ArrayList<>();
-		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x
-		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
-		data.add(new Tuple3<>(1, new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
-		// POJO is not initialized according to the first two fields.
-		data.add(new Tuple3<>(2, new CrazyNested("two", "duo", 2L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x
-		return env.fromCollection(data);
-	}
-
-	public static class Pojo1 {
-		public String a;
-		public String b;
-
-		public Pojo1() {}
-
-		public Pojo1(String a, String b) {
-			this.a = a;
-			this.b = b;
-		}
-	}
-
-	public static class Pojo2 {
-		public String a2;
-		public String b2;
-	}
-
-	public static class PojoWithMultiplePojos {
-		public Pojo1 p1;
-		public Pojo2 p2;
-		public Integer i0;
-
-		public PojoWithMultiplePojos() {
-		}
-
-		public PojoWithMultiplePojos(String a, String b, String a1, String b1, Integer i0) {
-			p1 = new Pojo1();
-			p1.a = a;
-			p1.b = b;
-			p2 = new Pojo2();
-			p2.a2 = a1;
-			p2.b2 = b1;
-			this.i0 = i0;
-		}
-	}
-
-	public static DataSet<PojoWithMultiplePojos> getPojoWithMultiplePojos(ExecutionEnvironment env) {
-		List<PojoWithMultiplePojos> data = new ArrayList<>();
-		data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
-		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
-		return env.fromCollection(data);
-	}
-
-	public enum Category {
-		CAT_A, CAT_B
-	}
-
-	public static class PojoWithDateAndEnum {
-		public String group;
-		public Date date;
-		public Category cat;
-	}
-	
-	public static DataSet<PojoWithDateAndEnum> getPojoWithDateAndEnum(ExecutionEnvironment env) {
-		List<PojoWithDateAndEnum> data = new ArrayList<>();
-		
-		PojoWithDateAndEnum one = new PojoWithDateAndEnum();
-		one.group = "a"; one.date = new Date(666); one.cat = Category.CAT_A;
-		data.add(one);
-		
-		PojoWithDateAndEnum two = new PojoWithDateAndEnum();
-		two.group = "a"; two.date = new Date(666); two.cat = Category.CAT_A;
-		data.add(two);
-		
-		PojoWithDateAndEnum three = new PojoWithDateAndEnum();
-		three.group = "b"; three.date = new Date(666); three.cat = Category.CAT_B;
-		data.add(three);
-		
-		return env.fromCollection(data);
-	}
-
-	public static class PojoWithCollection {
-		public List<Pojo1> pojos;
-		public int key;
-		public java.sql.Date sqlDate;
-		public BigInteger bigInt;
-		public BigDecimal bigDecimalKeepItNull;
-		public BigInt scalaBigInt;
-		public List<Object> mixed;
-
-		@Override
-		public String toString() {
-			return "PojoWithCollection{" +
-					"pojos.size()=" + pojos.size() +
-					", key=" + key +
-					", sqlDate=" + sqlDate +
-					", bigInt=" + bigInt +
-					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
-					", scalaBigInt=" + scalaBigInt +
-					", mixed=" + mixed +
-					'}';
-		}
-	}
-
-	public static class PojoWithCollectionGeneric {
-		public List<Pojo1> pojos;
-		public int key;
-		public java.sql.Date sqlDate;
-		public BigInteger bigInt;
-		public BigDecimal bigDecimalKeepItNull;
-		public BigInt scalaBigInt;
-		public List<Object> mixed;
-		private PojoWithDateAndEnum makeMeGeneric;
-
-		@Override
-		public String toString() {
-			return "PojoWithCollection{" +
-					"pojos.size()=" + pojos.size() +
-					", key=" + key +
-					", sqlDate=" + sqlDate +
-					", bigInt=" + bigInt +
-					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
-					", scalaBigInt=" + scalaBigInt +
-					", mixed=" + mixed +
-					'}';
-		}
-	}
-
-	public static DataSet<PojoWithCollection> getPojoWithCollection(ExecutionEnvironment env) {
-		List<PojoWithCollection> data = new ArrayList<>();
-
-		List<Pojo1> pojosList1 = new ArrayList<>();
-		pojosList1.add(new Pojo1("a", "aa"));
-		pojosList1.add(new Pojo1("b", "bb"));
-
-		List<Pojo1> pojosList2 = new ArrayList<>();
-		pojosList2.add(new Pojo1("a2", "aa2"));
-		pojosList2.add(new Pojo1("b2", "bb2"));
-
-		PojoWithCollection pwc1 = new PojoWithCollection();
-		pwc1.pojos = pojosList1;
-		pwc1.key = 0;
-		pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
-		pwc1.scalaBigInt = BigInt.int2bigInt(10);
-		pwc1.bigDecimalKeepItNull = null;
-		
-		// use calendar to make it stable across time zones
-		GregorianCalendar gcl1 = new GregorianCalendar(2033, 4, 18);
-		pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis());
-		pwc1.mixed = new ArrayList<>();
-		Map<String, Integer> map = new HashMap<>();
-		map.put("someKey", 1); // map.put("anotherKey", 2); map.put("third", 3);
-		pwc1.mixed.add(map);
-		pwc1.mixed.add(new File("/this/is/wrong"));
-		pwc1.mixed.add("uhlala");
-
-		PojoWithCollection pwc2 = new PojoWithCollection();
-		pwc2.pojos = pojosList2;
-		pwc2.key = 0;
-		pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
-		pwc2.scalaBigInt = BigInt.int2bigInt(31104000);
-		pwc2.bigDecimalKeepItNull = null;
-		
-		GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3);
-		pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976
-
-
-		data.add(pwc1);
-		data.add(pwc2);
-
-		return env.fromCollection(data);
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
deleted file mode 100644
index 04a7bc5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/util/ValueCollectionDataSets.java
+++ /dev/null
@@ -1,730 +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.javaApiOperators.util;
-
-import java.io.File;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.ValueTypeInfo;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.StringValue;
-import org.apache.hadoop.io.IntWritable;
-
-import scala.math.BigInt;
-
-/**
- * #######################################################################################################
- * 
- * 			BE AWARE THAT OTHER TESTS DEPEND ON THIS TEST DATA. 
- * 			IF YOU MODIFY THE DATA MAKE SURE YOU CHECK THAT ALL TESTS ARE STILL WORKING!
- * 
- * #######################################################################################################
- */
-public class ValueCollectionDataSets {
-
-	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> get3TupleDataSet(ExecutionEnvironment env) {
-		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
-
-		data.add(new Tuple3<>(new IntValue(1), new LongValue(1l), new StringValue("Hi")));
-		data.add(new Tuple3<>(new IntValue(2), new LongValue(2l), new StringValue("Hello")));
-		data.add(new Tuple3<>(new IntValue(3), new LongValue(2l), new StringValue("Hello world")));
-		data.add(new Tuple3<>(new IntValue(4), new LongValue(3l), new StringValue("Hello world, how are you?")));
-		data.add(new Tuple3<>(new IntValue(5), new LongValue(3l), new StringValue("I am fine.")));
-		data.add(new Tuple3<>(new IntValue(6), new LongValue(3l), new StringValue("Luke Skywalker")));
-		data.add(new Tuple3<>(new IntValue(7), new LongValue(4l), new StringValue("Comment#1")));
-		data.add(new Tuple3<>(new IntValue(8), new LongValue(4l), new StringValue("Comment#2")));
-		data.add(new Tuple3<>(new IntValue(9), new LongValue(4l), new StringValue("Comment#3")));
-		data.add(new Tuple3<>(new IntValue(10), new LongValue(4l), new StringValue("Comment#4")));
-		data.add(new Tuple3<>(new IntValue(11), new LongValue(5l), new StringValue("Comment#5")));
-		data.add(new Tuple3<>(new IntValue(12), new LongValue(5l), new StringValue("Comment#6")));
-		data.add(new Tuple3<>(new IntValue(13), new LongValue(5l), new StringValue("Comment#7")));
-		data.add(new Tuple3<>(new IntValue(14), new LongValue(5l), new StringValue("Comment#8")));
-		data.add(new Tuple3<>(new IntValue(15), new LongValue(5l), new StringValue("Comment#9")));
-		data.add(new Tuple3<>(new IntValue(16), new LongValue(6l), new StringValue("Comment#10")));
-		data.add(new Tuple3<>(new IntValue(17), new LongValue(6l), new StringValue("Comment#11")));
-		data.add(new Tuple3<>(new IntValue(18), new LongValue(6l), new StringValue("Comment#12")));
-		data.add(new Tuple3<>(new IntValue(19), new LongValue(6l), new StringValue("Comment#13")));
-		data.add(new Tuple3<>(new IntValue(20), new LongValue(6l), new StringValue("Comment#14")));
-		data.add(new Tuple3<>(new IntValue(21), new LongValue(6l), new StringValue("Comment#15")));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple3<IntValue, LongValue, StringValue>> getSmall3TupleDataSet(ExecutionEnvironment env) {
-		List<Tuple3<IntValue, LongValue, StringValue>> data = new ArrayList<>();
-
-		data.add(new Tuple3<>(new IntValue(1), new LongValue(1l), new StringValue("Hi")));
-		data.add(new Tuple3<>(new IntValue(2), new LongValue(2l), new StringValue("Hello")));
-		data.add(new Tuple3<>(new IntValue(3), new LongValue(2l), new StringValue("Hello world")));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> get5TupleDataSet(ExecutionEnvironment env) {
-		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
-
-		data.add(new Tuple5<>(new IntValue(1), new LongValue(1l), new IntValue(0), new StringValue("Hallo"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(2), new LongValue(2l), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(2), new LongValue(3l), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(3), new LongValue(4l), new IntValue(3), new StringValue("Hallo Welt wie gehts?"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(3), new LongValue(5l), new IntValue(4), new StringValue("ABC"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(3), new LongValue(6l), new IntValue(5), new StringValue("BCD"), new LongValue(3l)));
-		data.add(new Tuple5<>(new IntValue(4), new LongValue(7l), new IntValue(6), new StringValue("CDE"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(4), new LongValue(8l), new IntValue(7), new StringValue("DEF"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(4), new LongValue(9l), new IntValue(8), new StringValue("EFG"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(4), new LongValue(10l), new IntValue(9), new StringValue("FGH"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(5), new LongValue(11l), new IntValue(10), new StringValue("GHI"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(5), new LongValue(12l), new IntValue(11), new StringValue("HIJ"), new LongValue(3l)));
-		data.add(new Tuple5<>(new IntValue(5), new LongValue(13l), new IntValue(12), new StringValue("IJK"), new LongValue(3l)));
-		data.add(new Tuple5<>(new IntValue(5), new LongValue(14l), new IntValue(13), new StringValue("JKL"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(5), new LongValue(15l), new IntValue(14), new StringValue("KLM"), new LongValue(2l)));
-
-		Collections.shuffle(data);
-
-		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
-			TupleTypeInfo<>(
-				ValueTypeInfo.INT_VALUE_TYPE_INFO,
-				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
-				ValueTypeInfo.INT_VALUE_TYPE_INFO,
-				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
-				ValueTypeInfo.LONG_VALUE_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> getSmall5TupleDataSet(ExecutionEnvironment env) {
-		List<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> data = new ArrayList<>();
-
-		data.add(new Tuple5<>(new IntValue(1), new LongValue(1l), new IntValue(0), new StringValue("Hallo"), new LongValue(1l)));
-		data.add(new Tuple5<>(new IntValue(2), new LongValue(2l), new IntValue(1), new StringValue("Hallo Welt"), new LongValue(2l)));
-		data.add(new Tuple5<>(new IntValue(2), new LongValue(3l), new IntValue(2), new StringValue("Hallo Welt wie"), new LongValue(1l)));
-
-		Collections.shuffle(data);
-
-		TupleTypeInfo<Tuple5<IntValue, LongValue, IntValue, StringValue, LongValue>> type = new
-			TupleTypeInfo<>(
-				ValueTypeInfo.INT_VALUE_TYPE_INFO,
-				ValueTypeInfo.LONG_VALUE_TYPE_INFO,
-				ValueTypeInfo.INT_VALUE_TYPE_INFO,
-				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
-				ValueTypeInfo.LONG_VALUE_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getSmallNestedTupleDataSet(ExecutionEnvironment env) {
-		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
-
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(1)), new StringValue("one")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("two")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("three")));
-
-		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
-			TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
-				ValueTypeInfo.STRING_VALUE_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> getGroupSortedNestedTupleDataSet(ExecutionEnvironment env) {
-		List<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> data = new ArrayList<>();
-
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(3)), new StringValue("a")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(1), new IntValue(2)), new StringValue("a")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(1)), new StringValue("a")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(2), new IntValue(2)), new StringValue("b")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(3)), new StringValue("c")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(3), new IntValue(6)), new StringValue("c")));
-		data.add(new Tuple2<>(new Tuple2<>(new IntValue(4), new IntValue(9)), new StringValue("c")));
-
-		TupleTypeInfo<Tuple2<Tuple2<IntValue, IntValue>, StringValue>> type = new
-			TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
-				ValueTypeInfo.STRING_VALUE_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-
-	public static DataSet<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> getGroupSortedNestedTupleDataSet2(ExecutionEnvironment env) {
-		List<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> data = new ArrayList<>();
-
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(3)), new StringValue("a"), new IntValue(2)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(1), new IntValue(2)), new StringValue("a"), new IntValue(1)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(1)), new StringValue("a"), new IntValue(3)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(2), new IntValue(2)), new StringValue("b"), new IntValue(4)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(3)), new StringValue("c"), new IntValue(5)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(3), new IntValue(6)), new StringValue("c"), new IntValue(6)));
-		data.add(new Tuple3<>(new Tuple2<IntValue, IntValue>(new IntValue(4), new IntValue(9)), new StringValue("c"), new IntValue(7)));
-
-		TupleTypeInfo<Tuple3<Tuple2<IntValue, IntValue>, StringValue, IntValue>> type = new
-			TupleTypeInfo<>(
-				new TupleTypeInfo<Tuple2<IntValue, IntValue>>(ValueTypeInfo.INT_VALUE_TYPE_INFO, ValueTypeInfo.INT_VALUE_TYPE_INFO),
-				ValueTypeInfo.STRING_VALUE_TYPE_INFO,
-				ValueTypeInfo.INT_VALUE_TYPE_INFO
-		);
-
-		return env.fromCollection(data, type);
-	}
-	
-	public static DataSet<StringValue> getStringDataSet(ExecutionEnvironment env) {
-		List<StringValue> data = new ArrayList<>();
-
-		data.add(new StringValue("Hi"));
-		data.add(new StringValue("Hello"));
-		data.add(new StringValue("Hello world"));
-		data.add(new StringValue("Hello world, how are you?"));
-		data.add(new StringValue("I am fine."));
-		data.add(new StringValue("Luke Skywalker"));
-		data.add(new StringValue("Random comment"));
-		data.add(new StringValue("LOL"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<IntValue> getIntDataSet(ExecutionEnvironment env) {
-		List<IntValue> data = new ArrayList<>();
-
-		data.add(new IntValue(1));
-		data.add(new IntValue(2));
-		data.add(new IntValue(2));
-		data.add(new IntValue(3));
-		data.add(new IntValue(3));
-		data.add(new IntValue(3));
-		data.add(new IntValue(4));
-		data.add(new IntValue(4));
-		data.add(new IntValue(4));
-		data.add(new IntValue(4));
-		data.add(new IntValue(5));
-		data.add(new IntValue(5));
-		data.add(new IntValue(5));
-		data.add(new IntValue(5));
-		data.add(new IntValue(5));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<CustomType> getCustomTypeDataSet(ExecutionEnvironment env) {
-		List<CustomType> data = new ArrayList<CustomType>();
-
-		data.add(new CustomType(1, 0l, "Hi"));
-		data.add(new CustomType(2, 1l, "Hello"));
-		data.add(new CustomType(2, 2l, "Hello world"));
-		data.add(new CustomType(3, 3l, "Hello world, how are you?"));
-		data.add(new CustomType(3, 4l, "I am fine."));
-		data.add(new CustomType(3, 5l, "Luke Skywalker"));
-		data.add(new CustomType(4, 6l, "Comment#1"));
-		data.add(new CustomType(4, 7l, "Comment#2"));
-		data.add(new CustomType(4, 8l, "Comment#3"));
-		data.add(new CustomType(4, 9l, "Comment#4"));
-		data.add(new CustomType(5, 10l, "Comment#5"));
-		data.add(new CustomType(5, 11l, "Comment#6"));
-		data.add(new CustomType(5, 12l, "Comment#7"));
-		data.add(new CustomType(5, 13l, "Comment#8"));
-		data.add(new CustomType(5, 14l, "Comment#9"));
-		data.add(new CustomType(6, 15l, "Comment#10"));
-		data.add(new CustomType(6, 16l, "Comment#11"));
-		data.add(new CustomType(6, 17l, "Comment#12"));
-		data.add(new CustomType(6, 18l, "Comment#13"));
-		data.add(new CustomType(6, 19l, "Comment#14"));
-		data.add(new CustomType(6, 20l, "Comment#15"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<CustomType> getSmallCustomTypeDataSet(ExecutionEnvironment env) {
-		List<CustomType> data = new ArrayList<CustomType>();
-
-		data.add(new CustomType(1, 0l, "Hi"));
-		data.add(new CustomType(2, 1l, "Hello"));
-		data.add(new CustomType(2, 2l, "Hello world"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static class CustomType implements Serializable {
-
-		private static final long serialVersionUID = 1L;
-
-		public IntValue myInt;
-		public LongValue myLong;
-		public StringValue myString;
-
-		public CustomType() {
-		}
-
-		public CustomType(int i, long l, String s) {
-			myInt = new IntValue(i);
-			myLong = new LongValue(l);
-			myString = new StringValue(s);
-		}
-
-		@Override
-		public String toString() {
-			return myInt + "," + myLong + "," + myString;
-		}
-	}
-
-	public static class CustomTypeComparator implements Comparator<CustomType> {
-
-		@Override
-		public int compare(CustomType o1, CustomType o2) {
-			int diff = o1.myInt.getValue() - o2.myInt.getValue();
-			if (diff != 0) {
-				return diff;
-			}
-			diff = (int) (o1.myLong.getValue() - o2.myLong.getValue());
-			return diff != 0 ? diff : o1.myString.getValue().compareTo(o2.myString.getValue());
-		}
-
-	}
-
-	public static DataSet<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> getSmallTuplebasedDataSet(ExecutionEnvironment env) {
-		List<Tuple7<IntValue, StringValue, IntValue, IntValue, LongValue, StringValue, LongValue>> data = new ArrayList<>();
-		
-		data.add(new Tuple7<>(new IntValue(1), new StringValue("First"), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new LongValue(10000L)));
-		data.add(new Tuple7<>(new IntValue(2), new StringValue("Second"), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new LongValue(20000L)));
-		data.add(new Tuple7<>(new IntValue(3), new StringValue("Third"), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new LongValue(30000L)));
-
-		return env.fromCollection(data);
-	}
-	
-	public static DataSet<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> getSmallTuplebasedDataSetMatchingPojo(ExecutionEnvironment env) {
-		List<Tuple7<LongValue, IntValue, IntValue, LongValue, StringValue, IntValue, StringValue>> data = new ArrayList<>();
-		
-		data.add(new Tuple7<>(new LongValue(10000L), new IntValue(10), new IntValue(100), new LongValue(1000L), new StringValue("One"), new IntValue(1), new StringValue("First")));
-		data.add(new Tuple7<>(new LongValue(20000L), new IntValue(20), new IntValue(200), new LongValue(2000L), new StringValue("Two"), new IntValue(2), new StringValue("Second")));
-		data.add(new Tuple7<>(new LongValue(30000L), new IntValue(30), new IntValue(300), new LongValue(3000L), new StringValue("Three"), new IntValue(3), new StringValue("Third")));
-		
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getSmallPojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<POJO>();
-
-		data.add(new POJO(1 /*number*/, "First" /*str*/, 10 /*f0*/, 100/*f1.myInt*/, 1000L/*f1.myLong*/, "One" /*f1.myString*/, 10000L /*nestedPojo.longNumber*/));
-		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getDuplicatePojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<POJO>();
-
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L)); // 5x
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10000L));
-		data.add(new POJO(2, "Second", 20, 200, 2000L, "Two", 20000L));
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L)); // 2x
-		data.add(new POJO(3, "Third", 30, 300, 3000L, "Three", 30000L));
-
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<POJO> getMixedPojoDataSet(ExecutionEnvironment env) {
-		List<POJO> data = new ArrayList<POJO>();
-
-		data.add(new POJO(1, "First", 10, 100, 1000L, "One", 10100L)); // 5x
-		data.add(new POJO(2, "First_", 10, 105, 1000L, "One", 10200L));
-		data.add(new POJO(3, "First", 11, 102, 3000L, "One", 10200L));
-		data.add(new POJO(4, "First_", 11, 106, 1000L, "One", 10300L));
-		data.add(new POJO(5, "First", 11, 102, 2000L, "One", 10100L));
-		data.add(new POJO(6, "Second_", 20, 200, 2000L, "Two", 10100L));
-		data.add(new POJO(7, "Third", 31, 301, 2000L, "Three", 10200L)); // 2x
-		data.add(new POJO(8, "Third_", 30, 300, 1000L, "Three", 10100L));
-
-		return env.fromCollection(data);
-	}
-
-	public static class POJO {
-		public IntValue number;
-		public StringValue str;
-		public Tuple2<IntValue, CustomType> nestedTupleWithCustom;
-		public NestedPojo nestedPojo;
-		public transient LongValue ignoreMe;
-
-		public POJO(int i0, String s0,
-					int i1, int i2, long l0, String s1,
-					long l1) {
-			this.number = new IntValue(i0);
-			this.str = new StringValue(s0);
-			this.nestedTupleWithCustom = new Tuple2<>(new IntValue(i1), new CustomType(i2, l0, s1));
-			this.nestedPojo = new NestedPojo();
-			this.nestedPojo.longNumber = new LongValue(l1);
-		}
-
-		public POJO() {
-		}
-
-		@Override
-		public String toString() {
-			return number + " " + str + " " + nestedTupleWithCustom + " " + nestedPojo.longNumber;
-		}
-	}
-
-	public static class NestedPojo {
-		public static Object ignoreMe;
-		public LongValue longNumber;
-
-		public NestedPojo() {
-		}
-	}
-
-	public static DataSet<CrazyNested> getCrazyNestedDataSet(ExecutionEnvironment env) {
-		List<CrazyNested> data = new ArrayList<CrazyNested>();
-
-		data.add(new CrazyNested("aa"));
-		data.add(new CrazyNested("bb"));
-		data.add(new CrazyNested("bb"));
-		data.add(new CrazyNested("cc"));
-		data.add(new CrazyNested("cc"));
-		data.add(new CrazyNested("cc"));
-
-		return env.fromCollection(data);
-	}
-
-	public static class CrazyNested {
-		public CrazyNestedL1 nest_Lvl1;
-		public LongValue something; // test proper null-value handling
-
-		public CrazyNested() {
-		}
-
-		public CrazyNested(String set, String second, long s) { // additional CTor to set all fields to non-null values
-			this(set);
-			something = new LongValue(s);
-			nest_Lvl1.a = new StringValue(second);
-		}
-
-		public CrazyNested(String set) {
-			nest_Lvl1 = new CrazyNestedL1();
-			nest_Lvl1.nest_Lvl2 = new CrazyNestedL2();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3 = new CrazyNestedL3();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4 = new CrazyNestedL4();
-			nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal = new StringValue(set);
-		}
-	}
-
-	public static class CrazyNestedL1 {
-		public StringValue a;
-		public IntValue b;
-		public CrazyNestedL2 nest_Lvl2;
-	}
-
-	public static class CrazyNestedL2 {
-		public CrazyNestedL3 nest_Lvl3;
-	}
-
-	public static class CrazyNestedL3 {
-		public CrazyNestedL4 nest_Lvl4;
-	}
-
-	public static class CrazyNestedL4 {
-		public StringValue f1nal;
-	}
-
-	// Copied from TypeExtractorTest
-	public static class FromTuple extends Tuple3<StringValue, StringValue, LongValue> {
-		private static final long serialVersionUID = 1L;
-		public IntValue special;
-	}
-
-	public static class FromTupleWithCTor extends FromTuple {
-
-		private static final long serialVersionUID = 1L;
-
-		public FromTupleWithCTor() {}
-
-		public FromTupleWithCTor(int special, long tupleField) {
-			this.special = new IntValue(special);
-			this.setField(new LongValue(tupleField), 2);
-		}
-	}
-
-	public static DataSet<FromTupleWithCTor> getPojoExtendingFromTuple(ExecutionEnvironment env) {
-		List<FromTupleWithCTor> data = new ArrayList<>();
-		data.add(new FromTupleWithCTor(1, 10L)); // 3x
-		data.add(new FromTupleWithCTor(1, 10L));
-		data.add(new FromTupleWithCTor(1, 10L));
-		data.add(new FromTupleWithCTor(2, 20L)); // 2x
-		data.add(new FromTupleWithCTor(2, 20L));
-		return env.fromCollection(data);
-	}
-
-	public static class PojoContainingTupleAndWritable {
-		public IntValue someInt;
-		public StringValue someString;
-		public IntWritable hadoopFan;
-		public Tuple2<LongValue, LongValue> theTuple;
-
-		public PojoContainingTupleAndWritable() {
-		}
-
-		public PojoContainingTupleAndWritable(int i, long l1, long l2) {
-			hadoopFan = new IntWritable(i);
-			someInt = new IntValue(i);
-			theTuple = new Tuple2<>(new LongValue(l1), new LongValue(l2));
-		}
-	}
-
-	public static DataSet<PojoContainingTupleAndWritable> getPojoContainingTupleAndWritable(ExecutionEnvironment env) {
-		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
-		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L));
-		return env.fromCollection(data);
-	}
-
-
-
-	public static DataSet<PojoContainingTupleAndWritable> getGroupSortedPojoContainingTupleAndWritable(ExecutionEnvironment env) {
-		List<PojoContainingTupleAndWritable> data = new ArrayList<>();
-		data.add(new PojoContainingTupleAndWritable(1, 10L, 100L)); // 1x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 200L)); // 5x
-		data.add(new PojoContainingTupleAndWritable(2, 20L, 201L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 200L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 600L));
-		data.add(new PojoContainingTupleAndWritable(2, 30L, 400L));
-		return env.fromCollection(data);
-	}
-
-	public static DataSet<Tuple3<IntValue, CrazyNested, POJO>> getTupleContainingPojos(ExecutionEnvironment env) {
-		List<Tuple3<IntValue, CrazyNested, POJO>> data = new ArrayList<>();
-		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 3x
-		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
-		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(1), new CrazyNested("one", "uno", 1L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L)));
-		// POJO is not initialized according to the first two fields.
-		data.add(new Tuple3<IntValue, CrazyNested, POJO>(new IntValue(2), new CrazyNested("two", "duo", 2L), new POJO(1, "First", 10, 100, 1000L, "One", 10000L))); // 1x
-		return env.fromCollection(data);
-	}
-
-	public static class Pojo1 {
-		public StringValue a;
-		public StringValue b;
-
-		public Pojo1() {}
-
-		public Pojo1(String a, String b) {
-			this.a = new StringValue(a);
-			this.b = new StringValue(b);
-		}
-	}
-
-	public static class Pojo2 {
-		public StringValue a2;
-		public StringValue b2;
-	}
-
-	public static class PojoWithMultiplePojos {
-		public Pojo1 p1;
-		public Pojo2 p2;
-		public IntValue i0;
-
-		public PojoWithMultiplePojos() {
-		}
-
-		public PojoWithMultiplePojos(String a, String b, String a1, String b1, int i0) {
-			p1 = new Pojo1();
-			p1.a = new StringValue(a);
-			p1.b = new StringValue(b);
-			p2 = new Pojo2();
-			p2.a2 = new StringValue(a1);
-			p2.b2 = new StringValue(b1);
-			this.i0 = new IntValue(i0);
-		}
-	}
-
-	public static DataSet<PojoWithMultiplePojos> getPojoWithMultiplePojos(ExecutionEnvironment env) {
-		List<PojoWithMultiplePojos> data = new ArrayList<>();
-		data.add(new PojoWithMultiplePojos("a", "aa", "b", "bb", 1));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("b", "bb", "c", "cc", 2));
-		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
-		data.add(new PojoWithMultiplePojos("d", "dd", "e", "ee", 3));
-		return env.fromCollection(data);
-	}
-
-	public enum Category {
-		CAT_A, CAT_B;
-	}
-
-	public static class PojoWithDateAndEnum {
-		public StringValue group;
-		public Date date;
-		public Category cat;
-	}
-	
-	public static DataSet<PojoWithDateAndEnum> getPojoWithDateAndEnum(ExecutionEnvironment env) {
-		List<PojoWithDateAndEnum> data = new ArrayList<PojoWithDateAndEnum>();
-		
-		PojoWithDateAndEnum one = new PojoWithDateAndEnum();
-		one.group = new StringValue("a");
-		one.date = new Date(666);
-		one.cat = Category.CAT_A;
-		data.add(one);
-		
-		PojoWithDateAndEnum two = new PojoWithDateAndEnum();
-		two.group = new StringValue("a");
-		two.date = new Date(666);
-		two.cat = Category.CAT_A;
-		data.add(two);
-		
-		PojoWithDateAndEnum three = new PojoWithDateAndEnum();
-		three.group = new StringValue("b");
-		three.date = new Date(666);
-		three.cat = Category.CAT_B;
-		data.add(three);
-		
-		return env.fromCollection(data);
-	}
-
-	public static class PojoWithCollection {
-		public List<Pojo1> pojos;
-		public IntValue key;
-		public java.sql.Date sqlDate;
-		public BigInteger bigInt;
-		public BigDecimal bigDecimalKeepItNull;
-		public BigInt scalaBigInt;
-		public List<Object> mixed;
-
-		@Override
-		public String toString() {
-			return "PojoWithCollection{" +
-					"pojos.size()=" + pojos.size() +
-					", key=" + key +
-					", sqlDate=" + sqlDate +
-					", bigInt=" + bigInt +
-					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
-					", scalaBigInt=" + scalaBigInt +
-					", mixed=" + mixed +
-					'}';
-		}
-	}
-
-	public static class PojoWithCollectionGeneric {
-		public List<Pojo1> pojos;
-		public IntValue key;
-		public java.sql.Date sqlDate;
-		public BigInteger bigInt;
-		public BigDecimal bigDecimalKeepItNull;
-		public BigInt scalaBigInt;
-		public List<Object> mixed;
-		private PojoWithDateAndEnum makeMeGeneric;
-
-		@Override
-		public String toString() {
-			return "PojoWithCollection{" +
-					"pojos.size()=" + pojos.size() +
-					", key=" + key +
-					", sqlDate=" + sqlDate +
-					", bigInt=" + bigInt +
-					", bigDecimalKeepItNull=" + bigDecimalKeepItNull +
-					", scalaBigInt=" + scalaBigInt +
-					", mixed=" + mixed +
-					'}';
-		}
-	}
-
-	public static DataSet<PojoWithCollection> getPojoWithCollection(ExecutionEnvironment env) {
-		List<PojoWithCollection> data = new ArrayList<>();
-
-		List<Pojo1> pojosList1 = new ArrayList<>();
-		pojosList1.add(new Pojo1("a", "aa"));
-		pojosList1.add(new Pojo1("b", "bb"));
-
-		List<Pojo1> pojosList2 = new ArrayList<>();
-		pojosList2.add(new Pojo1("a2", "aa2"));
-		pojosList2.add(new Pojo1("b2", "bb2"));
-
-		PojoWithCollection pwc1 = new PojoWithCollection();
-		pwc1.pojos = pojosList1;
-		pwc1.key = new IntValue(0);
-		pwc1.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
-		pwc1.scalaBigInt = BigInt.int2bigInt(10);
-		pwc1.bigDecimalKeepItNull = null;
-		
-		// use calendar to make it stable across time zones
-		GregorianCalendar gcl1 = new GregorianCalendar(2033, 04, 18);
-		pwc1.sqlDate = new java.sql.Date(gcl1.getTimeInMillis());
-		pwc1.mixed = new ArrayList<Object>();
-		Map<StringValue, IntValue> map = new HashMap<>();
-		map.put(new StringValue("someKey"), new IntValue(1));
-		pwc1.mixed.add(map);
-		pwc1.mixed.add(new File("/this/is/wrong"));
-		pwc1.mixed.add("uhlala");
-
-		PojoWithCollection pwc2 = new PojoWithCollection();
-		pwc2.pojos = pojosList2;
-		pwc2.key = new IntValue(0);
-		pwc2.bigInt = BigInteger.valueOf(Long.MAX_VALUE).multiply(BigInteger.TEN);
-		pwc2.scalaBigInt = BigInt.int2bigInt(31104000);
-		pwc2.bigDecimalKeepItNull = null;
-		
-		GregorianCalendar gcl2 = new GregorianCalendar(1976, 4, 3);
-		pwc2.sqlDate = new java.sql.Date(gcl2.getTimeInMillis()); // 1976
-
-		data.add(pwc1);
-		data.add(pwc2);
-
-		return env.fromCollection(data);
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/CheckForbiddenMethodsUsage.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/CheckForbiddenMethodsUsage.java b/flink-tests/src/test/java/org/apache/flink/test/manual/CheckForbiddenMethodsUsage.java
index aabe7c0..6413a3b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/CheckForbiddenMethodsUsage.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/CheckForbiddenMethodsUsage.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.manual;
 
 import org.apache.flink.types.parser.FieldParserTest;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-
 import org.reflections.Reflections;
 import org.reflections.scanners.MemberUsageScanner;
 import org.reflections.util.ClasspathHelper;
@@ -40,7 +40,7 @@ import static org.junit.Assert.assertEquals;
 
 /**
  * Tests via reflection that certain methods are not called in Flink.
- * 
+ *
  * <p>Forbidden calls include:
  *   - Byte / String conversions that do not specify an explicit charset
  *     because they produce different results in different locales
@@ -116,11 +116,10 @@ public class CheckForbiddenMethodsUsage {
 			.addUrls(ClasspathHelper.forPackage("org.apache.flink"))
 			.addScanners(new MemberUsageScanner()));
 
-
 		for (ForbiddenCall forbiddenCall : forbiddenCalls) {
 			final Set<Member> methodUsages = forbiddenCall.getUsages(reflections);
 			methodUsages.removeAll(forbiddenCall.getExclusions());
-			assertEquals("Unexpected calls: " + methodUsages,0, methodUsages.size());
+			assertEquals("Unexpected calls: " + methodUsages, 0, methodUsages.size());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
index 0692196..f02cf1c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
@@ -40,23 +40,26 @@ import java.util.List;
 
 import static org.junit.Assert.fail;
 
+/**
+ * Manual test for growing hash tables.
+ */
 public class HashTableRecordWidthCombinations {
 
 	public static void main(String[] args) throws Exception {
 
 		@SuppressWarnings("unchecked")
-		final TypeSerializer<Tuple2<Long, byte[]>> buildSerializer = 
+		final TypeSerializer<Tuple2<Long, byte[]>> buildSerializer =
 				new TupleSerializer<Tuple2<Long, byte[]>>(
 						(Class<Tuple2<Long, byte[]>>) (Class<?>) Tuple2.class,
 						new TypeSerializer<?>[] { LongSerializer.INSTANCE, BytePrimitiveArraySerializer.INSTANCE });
-		
+
 		final TypeSerializer<Long> probeSerializer = LongSerializer.INSTANCE;
 
 		final TypeComparator<Tuple2<Long, byte[]>> buildComparator = new TupleComparator<Tuple2<Long, byte[]>>(
 				new int[] {0},
 				new TypeComparator<?>[] { new LongComparator(true) },
 				new TypeSerializer<?>[] { LongSerializer.INSTANCE });
-		
+
 		final TypeComparator<Long> probeComparator = new LongComparator(true);
 
 		final TypePairComparator<Long, Tuple2<Long, byte[]>> pairComparator = new TypePairComparator<Long, Tuple2<Long, byte[]>>() {
@@ -85,7 +88,7 @@ public class HashTableRecordWidthCombinations {
 		final IOManager ioMan = new IOManagerAsync();
 
 		try {
-			final int pageSize = 32*1024;
+			final int pageSize = 32 * 1024;
 			final int numSegments = 34;
 
 			for (int num = 3400; num < 3550; num++) {
@@ -151,7 +154,7 @@ public class HashTableRecordWidthCombinations {
 					try {
 						while (table.nextRecord()) {
 							MutableObjectIterator<Tuple2<Long, byte[]>> matches = table.getBuildSideIterator();
-							while (matches.next() != null);
+							while (matches.next() != null) {}
 						}
 					}
 					catch (RuntimeException e) {
@@ -176,11 +179,11 @@ public class HashTableRecordWidthCombinations {
 			ioMan.shutdown();
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
-	
+
 	private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
 		ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
 		for (int i = 0; i < numSegments; i++) {
@@ -188,7 +191,7 @@ public class HashTableRecordWidthCombinations {
 		}
 		return list;
 	}
-	
+
 	private static void checkNoTempFilesRemain(IOManager ioManager) {
 		for (File dir : ioManager.getSpillingDirectories()) {
 			for (String file : dir.list()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
index 9821b05..c69e6fd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
@@ -18,14 +18,6 @@
 
 package org.apache.flink.test.manual;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Random;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -41,13 +33,24 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.Assert;
 
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+/**
+ * Test {@link UnilateralSortMerger} on a large set of {@code String}.
+ */
 public class MassiveStringSorting {
 
 	private static final long SEED = 347569784659278346L;
-	
-	
+
 	public void testStringSorting() {
 		File input = null;
 		File sorted = null;
@@ -55,12 +58,12 @@ public class MassiveStringSorting {
 		try {
 			// the source file
 			input = generateFileWithStrings(300000, "http://some-uri.com/that/is/a/common/prefix/to/all");
-			
+
 			// the sorted file
 			sorted = File.createTempFile("sorted_strings", "txt");
-			
-			String[] command = {"/bin/bash","-c","export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
-			
+
+			String[] command = {"/bin/bash", "-c", "export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
+
 			Process p = null;
 			try {
 				p = Runtime.getRuntime().exec(command);
@@ -74,37 +77,37 @@ public class MassiveStringSorting {
 					p.destroy();
 				}
 			}
-			
+
 			// sort the data
 			UnilateralSortMerger<String> sorter = null;
 			BufferedReader reader = null;
 			BufferedReader verifyReader = null;
-			
+
 			try {
 				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
-					
+
 				TypeSerializer<String> serializer = StringSerializer.INSTANCE;
 				TypeComparator<String> comparator = new StringComparator(true);
-				
+
 				reader = new BufferedReader(new FileReader(input));
 				MutableObjectIterator<String> inputIterator = new StringReaderMutableObjectIterator(reader);
-				
+
 				sorter = new UnilateralSortMerger<String>(mm, ioMan, inputIterator, new DummyInvokable(),
 						new RuntimeSerializerFactory<String>(serializer, String.class), comparator, 1.0, 4, 0.8f,
 						true /* use large record handler */, false);
 
 				MutableObjectIterator<String> sortedData = sorter.getIterator();
-				
+
 				reader.close();
-				
+
 				// verify
 				verifyReader = new BufferedReader(new FileReader(sorted));
 				String next;
-				
+
 				while ((next = verifyReader.readLine()) != null) {
 					String nextFromStratoSort = sortedData.next("");
-					
+
 					Assert.assertNotNull(nextFromStratoSort);
 					Assert.assertEquals(next, nextFromStratoSort);
 				}
@@ -135,23 +138,23 @@ public class MassiveStringSorting {
 			}
 		}
 	}
-	
+
 	@SuppressWarnings("unchecked")
 	public void testStringTuplesSorting() {
-		final int NUM_STRINGS = 300000;
-		
+		final int numStrings = 300000;
+
 		File input = null;
 		File sorted = null;
 
 		try {
 			// the source file
-			input = generateFileWithStringTuples(NUM_STRINGS, "http://some-uri.com/that/is/a/common/prefix/to/all");
-			
+			input = generateFileWithStringTuples(numStrings, "http://some-uri.com/that/is/a/common/prefix/to/all");
+
 			// the sorted file
 			sorted = File.createTempFile("sorted_strings", "txt");
-			
-			String[] command = {"/bin/bash","-c","export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
-			
+
+			String[] command = {"/bin/bash", "-c", "export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
+
 			Process p = null;
 			try {
 				p = Runtime.getRuntime().exec(command);
@@ -165,33 +168,31 @@ public class MassiveStringSorting {
 					p.destroy();
 				}
 			}
-			
+
 			// sort the data
 			UnilateralSortMerger<Tuple2<String, String[]>> sorter = null;
 			BufferedReader reader = null;
 			BufferedReader verifyReader = null;
-			
+
 			try {
 				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
-					
-				TupleTypeInfo<Tuple2<String, String[]>> typeInfo = (TupleTypeInfo<Tuple2<String, String[]>>) 
+
+				TupleTypeInfo<Tuple2<String, String[]>> typeInfo = (TupleTypeInfo<Tuple2<String, String[]>>)
 						TypeInfoParser.<Tuple2<String, String[]>>parse("Tuple2<String, String[]>");
 
 				TypeSerializer<Tuple2<String, String[]>> serializer = typeInfo.createSerializer(new ExecutionConfig());
 				TypeComparator<Tuple2<String, String[]>> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0, new ExecutionConfig());
-				
+
 				reader = new BufferedReader(new FileReader(input));
 				MutableObjectIterator<Tuple2<String, String[]>> inputIterator = new StringTupleReaderMutableObjectIterator(reader);
-				
+
 				sorter = new UnilateralSortMerger<Tuple2<String, String[]>>(mm, ioMan, inputIterator, new DummyInvokable(),
 						new RuntimeSerializerFactory<Tuple2<String, String[]>>(serializer, (Class<Tuple2<String, String[]>>) (Class<?>) Tuple2.class), comparator, 1.0, 4, 0.8f,
 						true /* use large record handler */, false);
 
-				
-				
 				// use this part to verify that all if good when sorting in memory
-				
+
 //				List<MemorySegment> memory = mm.allocatePages(new DummyInvokable(), mm.computeNumberOfPages(1024*1024*1024));
 //				NormalizedKeySorter<Tuple2<String, String[]>> nks = new NormalizedKeySorter<Tuple2<String,String[]>>(serializer, comparator, memory);
 //
@@ -200,36 +201,36 @@ public class MassiveStringSorting {
 //					while ((wi = inputIterator.next(wi)) != null) {
 //						Assert.assertTrue(nks.write(wi));
 //					}
-//					
+//
 //					new QuickSort().sort(nks);
 //				}
-//				
+//
 //				MutableObjectIterator<Tuple2<String, String[]>> sortedData = nks.getIterator();
-				
+
 				MutableObjectIterator<Tuple2<String, String[]>> sortedData = sorter.getIterator();
 				reader.close();
-				
+
 				// verify
 				verifyReader = new BufferedReader(new FileReader(sorted));
 				MutableObjectIterator<Tuple2<String, String[]>> verifyIterator = new StringTupleReaderMutableObjectIterator(verifyReader);
-				
+
 				Tuple2<String, String[]> next = new Tuple2<String, String[]>("", new String[0]);
 				Tuple2<String, String[]> nextFromStratoSort = new Tuple2<String, String[]>("", new String[0]);
-				
+
 				int num = 0;
-				
+
 				while ((next = verifyIterator.next(next)) != null) {
 					num++;
-					
+
 					nextFromStratoSort = sortedData.next(nextFromStratoSort);
 					Assert.assertNotNull(nextFromStratoSort);
-					
+
 					Assert.assertEquals(next.f0, nextFromStratoSort.f0);
 					Assert.assertArrayEquals(next.f1, nextFromStratoSort.f1);
 				}
-				
+
 				Assert.assertNull(sortedData.next(nextFromStratoSort));
-				Assert.assertEquals(NUM_STRINGS, num);
+				Assert.assertEquals(numStrings, num);
 
 			}
 			finally {
@@ -260,15 +261,15 @@ public class MassiveStringSorting {
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	private static final class StringReaderMutableObjectIterator implements MutableObjectIterator<String> {
-		
+
 		private final BufferedReader reader;
 
 		public StringReaderMutableObjectIterator(BufferedReader reader) {
 			this.reader = reader;
 		}
-		
+
 		@Override
 		public String next(String reuse) throws IOException {
 			return reader.readLine();
@@ -279,22 +280,22 @@ public class MassiveStringSorting {
 			return reader.readLine();
 		}
 	}
-	
+
 	private static final class StringTupleReaderMutableObjectIterator implements MutableObjectIterator<Tuple2<String, String[]>> {
-		
+
 		private final BufferedReader reader;
 
 		public StringTupleReaderMutableObjectIterator(BufferedReader reader) {
 			this.reader = reader;
 		}
-		
+
 		@Override
 		public Tuple2<String, String[]> next(Tuple2<String, String[]> reuse) throws IOException {
 			String line = reader.readLine();
 			if (line == null) {
 				return null;
 			}
-			
+
 			String[] parts = line.split(" ");
 			reuse.f0 = parts[0];
 			reuse.f1 = parts;
@@ -306,31 +307,31 @@ public class MassiveStringSorting {
 			return next(new Tuple2<String, String[]>());
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private File generateFileWithStrings(int numStrings, String prefix) throws IOException {
 		final Random rnd = new Random(SEED);
-		
+
 		final StringBuilder bld = new StringBuilder();
 		final int resetValue = prefix.length();
-		
+
 		bld.append(prefix);
-		
+
 		File f = File.createTempFile("strings", "txt");
 		BufferedWriter wrt = null;
 		try {
 			wrt = new BufferedWriter(new FileWriter(f));
-		
-			for (int i = 0 ; i < numStrings; i++) {
+
+			for (int i = 0; i < numStrings; i++) {
 				bld.setLength(resetValue);
-				
+
 				int len = rnd.nextInt(20) + 300;
 				for (int k = 0; k < len; k++) {
 					char c = (char) (rnd.nextInt(80) + 40);
 					bld.append(c);
 				}
-				
+
 				String str = bld.toString();
 				wrt.write(str);
 				wrt.newLine();
@@ -338,52 +339,52 @@ public class MassiveStringSorting {
 		} finally {
 			wrt.close();
 		}
-		
+
 		return f;
 	}
-	
+
 	private File generateFileWithStringTuples(int numStrings, String prefix) throws IOException {
 		final Random rnd = new Random(SEED);
-		
+
 		final StringBuilder bld = new StringBuilder();
 
 		File f = File.createTempFile("strings", "txt");
 		BufferedWriter wrt = null;
 		try {
 			wrt = new BufferedWriter(new FileWriter(f));
-		
-			for (int i = 0 ; i < numStrings; i++) {
+
+			for (int i = 0; i < numStrings; i++) {
 				bld.setLength(0);
-				
+
 				int numComps = rnd.nextInt(5) + 1;
-				
+
 				for (int z = 0; z < numComps; z++) {
 					if (z > 0) {
 						bld.append(' ');
 					}
 					bld.append(prefix);
-				
+
 					int len = rnd.nextInt(20) + 10;
 					for (int k = 0; k < len; k++) {
 						char c = (char) (rnd.nextInt(80) + 40);
 						bld.append(c);
 					}
 				}
-				
+
 				String str = bld.toString();
-				
+
 				wrt.write(str);
 				wrt.newLine();
 			}
 		} finally {
 			wrt.close();
 		}
-		
+
 		return f;
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static void main(String[] args) {
 		new MassiveStringSorting().testStringSorting();
 		new MassiveStringSorting().testStringTuplesSorting();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
index 9e37b79..453aa14 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
@@ -18,14 +18,6 @@
 
 package org.apache.flink.test.manual;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Random;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -42,12 +34,24 @@ import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.StringValue;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.Assert;
 
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+/**
+ * Test {@link UnilateralSortMerger} on a large set of {@link StringValue}.
+ */
 public class MassiveStringValueSorting {
 
 	private static final long SEED = 347569784659278346L;
-	
+
 	public void testStringValueSorting() {
 		File input = null;
 		File sorted = null;
@@ -55,12 +59,12 @@ public class MassiveStringValueSorting {
 		try {
 			// the source file
 			input = generateFileWithStrings(300000, "http://some-uri.com/that/is/a/common/prefix/to/all");
-			
+
 			// the sorted file
 			sorted = File.createTempFile("sorted_strings", "txt");
-			
-			String[] command = {"/bin/bash","-c","export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
-			
+
+			String[] command = {"/bin/bash", "-c", "export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
+
 			Process p = null;
 			try {
 				p = Runtime.getRuntime().exec(command);
@@ -74,38 +78,38 @@ public class MassiveStringValueSorting {
 					p.destroy();
 				}
 			}
-			
+
 			// sort the data
 			UnilateralSortMerger<StringValue> sorter = null;
 			BufferedReader reader = null;
 			BufferedReader verifyReader = null;
-			
+
 			try {
 				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
-					
+
 				TypeSerializer<StringValue> serializer = new CopyableValueSerializer<StringValue>(StringValue.class);
 				TypeComparator<StringValue> comparator = new CopyableValueComparator<StringValue>(true, StringValue.class);
-				
+
 				reader = new BufferedReader(new FileReader(input));
 				MutableObjectIterator<StringValue> inputIterator = new StringValueReaderMutableObjectIterator(reader);
-				
+
 				sorter = new UnilateralSortMerger<StringValue>(mm, ioMan, inputIterator, new DummyInvokable(),
 						new RuntimeSerializerFactory<StringValue>(serializer, StringValue.class), comparator, 1.0, 4, 0.8f,
 						true /* use large record handler */, true);
 
 				MutableObjectIterator<StringValue> sortedData = sorter.getIterator();
-				
+
 				reader.close();
-				
+
 				// verify
 				verifyReader = new BufferedReader(new FileReader(sorted));
 				String nextVerify;
 				StringValue nextFromFlinkSort = new StringValue();
-				
+
 				while ((nextVerify = verifyReader.readLine()) != null) {
 					nextFromFlinkSort = sortedData.next(nextFromFlinkSort);
-					
+
 					Assert.assertNotNull(nextFromFlinkSort);
 					Assert.assertEquals(nextVerify, nextFromFlinkSort.getValue());
 				}
@@ -138,23 +142,23 @@ public class MassiveStringValueSorting {
 			}
 		}
 	}
-	
+
 	@SuppressWarnings("unchecked")
 	public void testStringValueTuplesSorting() {
-		final int NUM_STRINGS = 300000;
-		
+		final int numStrings = 300000;
+
 		File input = null;
 		File sorted = null;
 
 		try {
 			// the source file
-			input = generateFileWithStringTuples(NUM_STRINGS, "http://some-uri.com/that/is/a/common/prefix/to/all");
-			
+			input = generateFileWithStringTuples(numStrings, "http://some-uri.com/that/is/a/common/prefix/to/all");
+
 			// the sorted file
 			sorted = File.createTempFile("sorted_strings", "txt");
-			
-			String[] command = {"/bin/bash","-c","export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
-			
+
+			String[] command = {"/bin/bash", "-c", "export LC_ALL=\"C\" && cat \"" + input.getAbsolutePath() + "\" | sort > \"" + sorted.getAbsolutePath() + "\""};
+
 			Process p = null;
 			try {
 				p = Runtime.getRuntime().exec(command);
@@ -168,33 +172,31 @@ public class MassiveStringValueSorting {
 					p.destroy();
 				}
 			}
-			
+
 			// sort the data
 			UnilateralSortMerger<Tuple2<StringValue, StringValue[]>> sorter = null;
 			BufferedReader reader = null;
 			BufferedReader verifyReader = null;
-			
+
 			try {
 				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
-					
+
 				TupleTypeInfo<Tuple2<StringValue, StringValue[]>> typeInfo = (TupleTypeInfo<Tuple2<StringValue, StringValue[]>>)
 						TypeInfoParser.<Tuple2<StringValue, StringValue[]>>parse("Tuple2<org.apache.flink.types.StringValue, org.apache.flink.types.StringValue[]>");
 
 				TypeSerializer<Tuple2<StringValue, StringValue[]>> serializer = typeInfo.createSerializer(new ExecutionConfig());
 				TypeComparator<Tuple2<StringValue, StringValue[]>> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0, new ExecutionConfig());
-				
+
 				reader = new BufferedReader(new FileReader(input));
 				MutableObjectIterator<Tuple2<StringValue, StringValue[]>> inputIterator = new StringValueTupleReaderMutableObjectIterator(reader);
-				
+
 				sorter = new UnilateralSortMerger<Tuple2<StringValue, StringValue[]>>(mm, ioMan, inputIterator, new DummyInvokable(),
 						new RuntimeSerializerFactory<Tuple2<StringValue, StringValue[]>>(serializer, (Class<Tuple2<StringValue, StringValue[]>>) (Class<?>) Tuple2.class), comparator, 1.0, 4, 0.8f,
 						true /* use large record handler */, false);
 
-				
-				
 				// use this part to verify that all if good when sorting in memory
-				
+
 //				List<MemorySegment> memory = mm.allocatePages(new DummyInvokable(), mm.computeNumberOfPages(1024*1024*1024));
 //				NormalizedKeySorter<Tuple2<String, String[]>> nks = new NormalizedKeySorter<Tuple2<String,String[]>>(serializer, comparator, memory);
 //
@@ -203,36 +205,36 @@ public class MassiveStringValueSorting {
 //					while ((wi = inputIterator.next(wi)) != null) {
 //						Assert.assertTrue(nks.write(wi));
 //					}
-//					
+//
 //					new QuickSort().sort(nks);
 //				}
-//				
+//
 //				MutableObjectIterator<Tuple2<String, String[]>> sortedData = nks.getIterator();
-				
+
 				MutableObjectIterator<Tuple2<StringValue, StringValue[]>> sortedData = sorter.getIterator();
 				reader.close();
-				
+
 				// verify
 				verifyReader = new BufferedReader(new FileReader(sorted));
 				MutableObjectIterator<Tuple2<StringValue, StringValue[]>> verifyIterator = new StringValueTupleReaderMutableObjectIterator(verifyReader);
-				
+
 				Tuple2<StringValue, StringValue[]> nextVerify = new Tuple2<StringValue, StringValue[]>(new StringValue(), new StringValue[0]);
 				Tuple2<StringValue, StringValue[]> nextFromFlinkSort = new Tuple2<StringValue, StringValue[]>(new StringValue(), new StringValue[0]);
-				
+
 				int num = 0;
-				
+
 				while ((nextVerify = verifyIterator.next(nextVerify)) != null) {
 					num++;
-					
+
 					nextFromFlinkSort = sortedData.next(nextFromFlinkSort);
 					Assert.assertNotNull(nextFromFlinkSort);
-					
+
 					Assert.assertEquals(nextVerify.f0, nextFromFlinkSort.f0);
 					Assert.assertArrayEquals(nextVerify.f1, nextFromFlinkSort.f1);
 				}
-				
+
 				Assert.assertNull(sortedData.next(nextFromFlinkSort));
-				Assert.assertEquals(NUM_STRINGS, num);
+				Assert.assertEquals(numStrings, num);
 
 			}
 			finally {
@@ -265,23 +267,23 @@ public class MassiveStringValueSorting {
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	private static final class StringValueReaderMutableObjectIterator implements MutableObjectIterator<StringValue> {
-		
+
 		private final BufferedReader reader;
 
 		public StringValueReaderMutableObjectIterator(BufferedReader reader) {
 			this.reader = reader;
 		}
-		
+
 		@Override
 		public StringValue next(StringValue reuse) throws IOException {
 			String line = reader.readLine();
-			
+
 			if (line == null) {
 				return null;
 			}
-			
+
 			reuse.setValue(line);
 			return reuse;
 		}
@@ -291,30 +293,30 @@ public class MassiveStringValueSorting {
 			return next(new StringValue());
 		}
 	}
-	
+
 	private static final class StringValueTupleReaderMutableObjectIterator implements MutableObjectIterator<Tuple2<StringValue, StringValue[]>> {
-		
+
 		private final BufferedReader reader;
 
 		public StringValueTupleReaderMutableObjectIterator(BufferedReader reader) {
 			this.reader = reader;
 		}
-		
+
 		@Override
 		public Tuple2<StringValue, StringValue[]> next(Tuple2<StringValue, StringValue[]> reuse) throws IOException {
 			String line = reader.readLine();
 			if (line == null) {
 				return null;
 			}
-			
+
 			String[] parts = line.split(" ");
 			reuse.f0.setValue(parts[0]);
 			reuse.f1 = new StringValue[parts.length];
-			
+
 			for (int i = 0; i < parts.length; i++) {
 				reuse.f1[i] = new StringValue(parts[i]);
 			}
-			
+
 			return reuse;
 		}
 
@@ -323,31 +325,31 @@ public class MassiveStringValueSorting {
 			return next(new Tuple2<StringValue, StringValue[]>(new StringValue(), new StringValue[0]));
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	private File generateFileWithStrings(int numStrings, String prefix) throws IOException {
 		final Random rnd = new Random(SEED);
-		
+
 		final StringBuilder bld = new StringBuilder();
 		final int resetValue = prefix.length();
-		
+
 		bld.append(prefix);
-		
+
 		File f = File.createTempFile("strings", "txt");
 		BufferedWriter wrt = null;
 		try {
 			wrt = new BufferedWriter(new FileWriter(f));
-		
-			for (int i = 0 ; i < numStrings; i++) {
+
+			for (int i = 0; i < numStrings; i++) {
 				bld.setLength(resetValue);
-				
+
 				int len = rnd.nextInt(20) + 300;
 				for (int k = 0; k < len; k++) {
 					char c = (char) (rnd.nextInt(80) + 40);
 					bld.append(c);
 				}
-				
+
 				String str = bld.toString();
 				wrt.write(str);
 				wrt.newLine();
@@ -357,40 +359,40 @@ public class MassiveStringValueSorting {
 				wrt.close();
 			}
 		}
-		
+
 		return f;
 	}
-	
+
 	private File generateFileWithStringTuples(int numStrings, String prefix) throws IOException {
 		final Random rnd = new Random(SEED);
-		
+
 		final StringBuilder bld = new StringBuilder();
 
 		File f = File.createTempFile("strings", "txt");
 		BufferedWriter wrt = null;
 		try {
 			wrt = new BufferedWriter(new FileWriter(f));
-		
-			for (int i = 0 ; i < numStrings; i++) {
+
+			for (int i = 0; i < numStrings; i++) {
 				bld.setLength(0);
-				
+
 				int numComps = rnd.nextInt(5) + 1;
-				
+
 				for (int z = 0; z < numComps; z++) {
 					if (z > 0) {
 						bld.append(' ');
 					}
 					bld.append(prefix);
-				
+
 					int len = rnd.nextInt(20) + 10;
 					for (int k = 0; k < len; k++) {
 						char c = (char) (rnd.nextInt(80) + 40);
 						bld.append(c);
 					}
 				}
-				
+
 				String str = bld.toString();
-				
+
 				wrt.write(str);
 				wrt.newLine();
 			}
@@ -399,12 +401,12 @@ public class MassiveStringValueSorting {
 				wrt.close();
 			}
 		}
-		
+
 		return f;
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
-	
+
 	public static void main(String[] args) {
 		new MassiveStringValueSorting().testStringValueSorting();
 		new MassiveStringValueSorting().testStringValueTuplesSorting();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
index ee3b4b2..0b8fd1c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java
@@ -38,14 +38,14 @@ import static org.junit.Assert.fail;
  * with a parallelism of 100.
  */
 public class NotSoMiniClusterIterations {
-	
+
 	private static final int PARALLELISM = 100;
-	
+
 	public static void main(String[] args) {
 		if ((Runtime.getRuntime().maxMemory() >>> 20) < 5000) {
 			throw new RuntimeException("This test program needs to run with at least 5GB of heap space.");
 		}
-		
+
 		LocalFlinkMiniCluster cluster = null;
 
 		try {
@@ -55,7 +55,7 @@ public class NotSoMiniClusterIterations {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
 			config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 1000);
 			config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 8 * 1024);
-			
+
 			config.setInteger("taskmanager.net.server.numThreads", 1);
 			config.setInteger("taskmanager.net.client.numThreads", 1);
 


[19/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java
new file mode 100644
index 0000000..3dfb672
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountITCase.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.examples.java.wordcount.WordCount;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test {@link WordCount}.
+ */
+public class WordCountITCase extends JavaProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WordCount.main(new String[] {
+				"--input", textPath,
+				"--output", resultPath });
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java
new file mode 100644
index 0000000..3ee9e5e
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountNestedPOJOITCase.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.java;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+import java.util.Date;
+
+/**
+ * WordCount with nested POJO example.
+ */
+@SuppressWarnings("serial")
+public class WordCountNestedPOJOITCase extends JavaProgramTestBase implements Serializable {
+	private static final long serialVersionUID = 1L;
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<String> text = env.readTextFile(textPath);
+
+		DataSet<WC> counts = text
+				.flatMap(new Tokenizer())
+				.groupBy("complex.someTest")
+				.reduce(new ReduceFunction<WC>() {
+					private static final long serialVersionUID = 1L;
+					public WC reduce(WC value1, WC value2) {
+						return new WC(value1.complex.someTest, value1.count + value2.count);
+					}
+				});
+
+		counts.writeAsText(resultPath);
+
+		env.execute("WordCount with custom data types example");
+	}
+
+	private static final class Tokenizer implements FlatMapFunction<String, WC> {
+
+		@Override
+		public void flatMap(String value, Collector<WC> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new WC(token, 1));
+				}
+			}
+		}
+	}
+
+	/**
+	 * POJO with nested POJO.
+	 */
+	public static class WC { // is a pojo
+		public ComplexNestedClass complex; // is a pojo
+		public int count; // is a BasicType
+
+		public WC() {
+		}
+
+		public WC(String t, int c) {
+			this.count = c;
+			this.complex = new ComplexNestedClass();
+			this.complex.word = new Tuple3<Long, Long, String>(0L, 0L, "egal");
+			this.complex.date = new Date();
+			this.complex.someFloat = 0.0f;
+			this.complex.someNumber = 666;
+			this.complex.someTest = t;
+		}
+
+		@Override
+		public String toString() {
+			return this.complex.someTest + " " + count;
+		}
+	}
+
+	/**
+	 * Nested POJO.
+	 */
+	public static class ComplexNestedClass { // pojo
+		public static int ignoreStaticField;
+		public transient int ignoreTransientField;
+		public Date date; // generic type
+		public Integer someNumber; // BasicType
+		public float someFloat; // BasicType
+		public Tuple3<Long, Long, String> word; //Tuple Type with three basic types
+		public String someTest;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java
new file mode 100644
index 0000000..ab76c12
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSimplePOJOITCase.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.java;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+
+/**
+ * WordCount with simple POJO example.
+ */
+public class WordCountSimplePOJOITCase extends JavaProgramTestBase implements Serializable {
+	private static final long serialVersionUID = 1L;
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> text = env.readTextFile(textPath);
+
+		DataSet<WC> counts = text
+				.flatMap(new Tokenizer())
+				.groupBy("word")
+				.reduce(new ReduceFunction<WC>() {
+					private static final long serialVersionUID = 1L;
+
+					public WC reduce(WC value1, WC value2) {
+						return new WC(value1.word, value1.count + value2.count);
+					}
+				});
+
+		counts.writeAsText(resultPath);
+
+		env.execute("WordCount with custom data types example");
+	}
+
+	private static final class Tokenizer implements FlatMapFunction<String, WC> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<WC> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new WC(token, 1));
+				}
+			}
+		}
+	}
+
+	/**
+	 * POJO with word and count.
+	 */
+	public static class WC {
+		public WC() {}
+
+		public WC(String w, int c) {
+			word = w;
+			count = c;
+		}
+
+		public String word;
+		public int count;
+
+		@Override
+		public String toString() {
+			return word + " " + count;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java
new file mode 100644
index 0000000..688916c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassInterfacePOJOITCase.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.java;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+
+/**
+ * WordCount with subclass and interface example.
+ */
+@SuppressWarnings("serial")
+public class WordCountSubclassInterfacePOJOITCase extends JavaProgramTestBase implements Serializable {
+	private static final long serialVersionUID = 1L;
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<String> text = env.readTextFile(textPath);
+
+		DataSet<WCBase> counts = text
+				.flatMap(new Tokenizer())
+				.groupBy("word")
+				.reduce(new ReduceFunction<WCBase>() {
+					private static final long serialVersionUID = 1L;
+					public WCBase reduce(WCBase value1, WCBase value2) {
+						WC wc1 = (WC) value1;
+						WC wc2 = (WC) value2;
+						int c = wc1.secretCount.getCount() + wc2.secretCount.getCount();
+						wc1.secretCount.setCount(c);
+						return wc1;
+					}
+				})
+				.map(new MapFunction<WCBase, WCBase>() {
+					@Override
+					public WCBase map(WCBase value) throws Exception {
+						WC wc = (WC) value;
+						wc.count = wc.secretCount.getCount();
+						return wc;
+					}
+				});
+
+		counts.writeAsText(resultPath);
+
+		env.execute("WordCount with custom data types example");
+	}
+
+	private static final class Tokenizer implements FlatMapFunction<String, WCBase> {
+
+		@Override
+		public void flatMap(String value, Collector<WCBase> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new WC(token, 1));
+				}
+			}
+		}
+	}
+
+	/**
+	 * Abstract POJO.
+	 */
+	public abstract static class WCBase {
+		public String word;
+		public int count;
+
+		public WCBase(String w, int c) {
+			this.word = w;
+			this.count = c;
+		}
+
+		@Override
+		public String toString() {
+			return word + " " + count;
+		}
+	}
+
+	/**
+	 * POJO interface.
+	 */
+	public interface CrazyCounter {
+		int getCount();
+
+		void setCount(int c);
+	}
+
+	/**
+	 * Implementation of POJO interface.
+	 */
+	public static class CrazyCounterImpl implements CrazyCounter {
+		public int countz;
+
+		public CrazyCounterImpl() {
+		}
+
+		public CrazyCounterImpl(int c) {
+			this.countz = c;
+		}
+
+		@Override
+		public int getCount() {
+			return countz;
+		}
+
+		@Override
+		public void setCount(int c) {
+			this.countz = c;
+		}
+
+	}
+
+	/**
+	 * Subclass of abstract POJO.
+	 */
+	public static class WC extends WCBase {
+		public CrazyCounter secretCount;
+
+		public WC() {
+			super(null, 0);
+		}
+
+		public WC(String w, int c) {
+			super(w, 0);
+			this.secretCount = new CrazyCounterImpl(c);
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java
new file mode 100644
index 0000000..450af37
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountSubclassPOJOITCase.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+
+/**
+ * WordCount with custom data types example.
+ */
+@SuppressWarnings("serial")
+public class WordCountSubclassPOJOITCase extends JavaProgramTestBase implements Serializable {
+	private static final long serialVersionUID = 1L;
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<String> text = env.readTextFile(textPath);
+
+		DataSet<WCBase> counts = text
+				.flatMap(new Tokenizer())
+				.groupBy("word")
+				.reduce(new ReduceFunction<WCBase>() {
+					private static final long serialVersionUID = 1L;
+					public WCBase reduce(WCBase value1, WCBase value2) {
+						WC wc1 = (WC) value1;
+						WC wc2 = (WC) value2;
+						return new WC(value1.word, wc1.secretCount + wc2.secretCount);
+					}
+				})
+				.map(new MapFunction<WCBase, WCBase>() {
+					@Override
+					public WCBase map(WCBase value) throws Exception {
+						WC wc = (WC) value;
+						wc.count = wc.secretCount;
+						return wc;
+					}
+				});
+
+		counts.writeAsText(resultPath);
+
+		env.execute("WordCount with custom data types example");
+	}
+
+	private static final class Tokenizer implements FlatMapFunction<String, WCBase> {
+
+		@Override
+		public void flatMap(String value, Collector<WCBase> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new WC(token, 1));
+				}
+			}
+		}
+	}
+
+	/**
+	 * Abstract POJO.
+	 */
+	public abstract static class WCBase {
+		public String word;
+		public int count;
+
+		public WCBase(String w, int c) {
+			this.word = w;
+			this.count = c;
+		}
+
+		@Override
+		public String toString() {
+			return word + " " + count;
+		}
+	}
+
+	/**
+	 * Subclass of abstract POJO.
+	 */
+	public static class WC extends WCBase {
+
+		public int secretCount;
+
+		public WC() {
+			super(null, 0);
+		}
+
+		public WC(String w, int c) {
+			super(w, 0);
+			this.secretCount = c;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java
new file mode 100644
index 0000000..405026c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WordCountWithCollectionITCase.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.java;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.wordcount.WordCount;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import org.junit.Assert;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * WordCount with collection example.
+ */
+public class WordCountWithCollectionITCase extends JavaProgramTestBase {
+
+	private final List<Tuple2<String, Integer>> resultsCollected = new ArrayList<Tuple2<String, Integer>>();
+
+	@Override
+	protected void postSubmit() throws Exception {
+		String[] result = new String[resultsCollected.size()];
+		for (int i = 0; i < result.length; i++) {
+			result[i] = resultsCollected.get(i).toString();
+		}
+		Arrays.sort(result);
+
+		String[] expected = WordCountData.COUNTS_AS_TUPLES.split("\n");
+		Arrays.sort(expected);
+
+		Assert.assertEquals("Different number of lines in expected and obtained result.", expected.length, result.length);
+		Assert.assertArrayEquals(expected, result);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> text = env.fromElements(WordCountData.TEXT);
+		DataSet<Tuple2<String, Integer>> words = text.flatMap(new WordCount.Tokenizer());
+		DataSet<Tuple2<String, Integer>> result = words.groupBy(0).aggregate(Aggregations.SUM, 1);
+
+		result.output(new LocalCollectionOutputFormat<Tuple2<String, Integer>>(resultsCollected));
+		env.execute("Word Count Collection");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/ConnectedComponentsITCase.java
new file mode 100644
index 0000000..0391556
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/ConnectedComponentsITCase.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.scala;
+
+import org.apache.flink.examples.scala.graph.ConnectedComponents;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.BufferedReader;
+
+/**
+ * Test for {@link ConnectedComponents}.
+ */
+public class ConnectedComponentsITCase extends JavaProgramTestBase {
+
+	private static final long SEED = 0xBADC0FFEEBEEFL;
+
+	private static final int NUM_VERTICES = 1000;
+
+	private static final int NUM_EDGES = 10000;
+
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+
+	@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 void testProgram() throws Exception {
+		ConnectedComponents.main(new String[] {
+				"--vertices", verticesPath,
+				"--edges", edgesPath,
+				"--output", resultPath,
+				"--iterations", "100"});
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			ConnectedComponentsData.checkOddEvenResult(reader);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/EnumTriangleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/EnumTriangleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/EnumTriangleITCase.java
new file mode 100644
index 0000000..7f9ac0e
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/EnumTriangleITCase.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.scala;
+
+import org.apache.flink.examples.scala.graph.EnumTriangles;
+import org.apache.flink.test.testdata.EnumTriangleData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test {@link EnumTriangles}.
+ */
+public class EnumTriangleITCase extends JavaProgramTestBase {
+
+	protected String edgePath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
+		resultPath = getTempDirPath("triangles");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		EnumTriangles.main(new String[] {
+				"--edges", edgePath,
+				"--output", resultPath });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
new file mode 100644
index 0000000..93c4f56
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/PageRankITCase.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.scala;
+
+import org.apache.flink.examples.scala.graph.PageRankBasic;
+import org.apache.flink.test.testdata.PageRankData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+
+/**
+ * Test for {@link PageRankBasic}.
+ */
+@RunWith(Parameterized.class)
+public class PageRankITCase extends MultipleProgramsTestBase {
+
+	public PageRankITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Before
+	public void before() throws Exception{
+		File resultFile = tempFolder.newFile();
+		//Delete file because the Scala API does not respect WriteMode set by the configuration
+		resultFile.delete();
+		resultPath = resultFile.toURI().toString();
+
+		File verticesFile = tempFolder.newFile();
+		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
+
+		File edgesFile = tempFolder.newFile();
+		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
+
+		verticesPath = verticesFile.toURI().toString();
+		edgesPath = edgesFile.toURI().toString();
+	}
+
+	@After
+	public void after() throws Exception{
+		compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01);
+	}
+
+	@Test
+	public void testPageRankWithSmallNumberOfIterations() throws Exception {
+		PageRankBasic.main(new String[] {
+				"--pages", verticesPath,
+				"--links", edgesPath,
+				"--output", resultPath,
+				"--numPages", PageRankData.NUM_VERTICES + "",
+				"--iterations", "3"});
+		expected = PageRankData.RANKS_AFTER_3_ITERATIONS;
+	}
+
+	@Test
+	public void testPageRankWithConvergence() throws Exception {
+		// start with a very high number of iteration such that the dynamic convergence criterion must handle termination
+		PageRankBasic.main(new String[] {
+				"--pages", verticesPath,
+				"--links", edgesPath,
+				"--output", resultPath,
+				"--numPages", PageRankData.NUM_VERTICES + "",
+				"--iterations", "1000"});
+		expected = PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/TransitiveClosureITCase.java
new file mode 100644
index 0000000..3963801
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/TransitiveClosureITCase.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.scala;
+
+import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.testdata.TransitiveClosureData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.BufferedReader;
+
+/**
+ * Test for {@link TransitiveClosureNaive}.
+ */
+public class TransitiveClosureITCase extends JavaProgramTestBase {
+
+	private static final long SEED = 0xBADC0FFEEBEEFL;
+
+	private static final int NUM_VERTICES = 100;
+
+	private static final int NUM_EDGES = 500;
+
+	private String edgesPath;
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
+		resultPath = getTempFilePath("results");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TransitiveClosureNaive.main(new String [] {
+				"--edges", edgesPath,
+				"--output", resultPath,
+				"--iterations", "5"});
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			TransitiveClosureData.checkOddEvenResult(reader);
+		}
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/WebLogAnalysisITCase.java
new file mode 100644
index 0000000..962a8a0
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/WebLogAnalysisITCase.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.scala;
+
+import org.apache.flink.examples.scala.relational.WebLogAnalysis;
+import org.apache.flink.test.testdata.WebLogAnalysisData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test for {@link WebLogAnalysis}.
+ */
+public class WebLogAnalysisITCase extends JavaProgramTestBase {
+
+	private String docsPath;
+	private String ranksPath;
+	private String visitsPath;
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		docsPath = createTempFile("docs", WebLogAnalysisData.DOCS);
+		ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS);
+		visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WebLogAnalysis.main(new String[]{
+				"--documents", docsPath,
+				"--ranks", ranksPath,
+				"--visits", visitsPath,
+				"--output", resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/scala/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/scala/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/scala/WordCountITCase.java
new file mode 100644
index 0000000..d7a00b2
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/scala/WordCountITCase.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.scala;
+
+import org.apache.flink.examples.scala.wordcount.WordCount;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test {@link WordCount}.
+ */
+public class WordCountITCase extends JavaProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	public WordCountITCase(){
+		setParallelism(4);
+		setNumTaskManagers(2);
+		setTaskManagerNumSlots(2);
+	}
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WordCount.main(new String[] {
+				"--input", textPath,
+				"--output", resultPath });
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.java
deleted file mode 100644
index d5b6b38..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/ConnectedComponentsITCase.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.exampleJavaPrograms;
-
-import java.io.BufferedReader;
-
-import org.apache.flink.examples.java.graph.ConnectedComponents;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class ConnectedComponentsITCase extends JavaProgramTestBase {
-	
-	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
-	private static final int NUM_VERTICES = 1000;
-	
-	private static final int NUM_EDGES = 10000;
-
-	
-	private String verticesPath;
-	private String edgesPath;
-	private String resultPath;
-	
-	
-	@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 void testProgram() throws Exception {
-		ConnectedComponents.main(
-				"--vertices", verticesPath,
-				"--edges", edgesPath,
-				"--output", resultPath,
-				"--iterations", "100");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		for (BufferedReader reader : getResultReader(resultPath)) {
-			ConnectedComponentsData.checkOddEvenResult(reader);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
deleted file mode 100644
index 4d61b37..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/EnumTriangleBasicITCase.java
+++ /dev/null
@@ -1,48 +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.exampleJavaPrograms;
-
-import org.apache.flink.examples.java.graph.EnumTriangles;
-import org.apache.flink.test.testdata.EnumTriangleData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class EnumTriangleBasicITCase extends JavaProgramTestBase {
-	
-	protected String edgePath;
-	protected String resultPath;
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
-		resultPath = getTempDirPath("triangles");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath);
-	}
-	
-	@Override
-	protected void testProgram() throws Exception {
-		EnumTriangles.main(new String[] {
-				"--edges", edgePath,
-				"--output", resultPath });
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
deleted file mode 100644
index efd534c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/PageRankITCase.java
+++ /dev/null
@@ -1,92 +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.exampleJavaPrograms;
-
-import java.io.File;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-import org.apache.flink.examples.java.graph.PageRank;
-import org.apache.flink.test.testdata.PageRankData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class PageRankITCase extends MultipleProgramsTestBase {
-
-	public PageRankITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	private String verticesPath;
-	private String edgesPath;
-	private String resultPath;
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception{
-		resultPath = tempFolder.newFile().toURI().toString();
-		File verticesFile = tempFolder.newFile();
-		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
-
-		verticesPath = verticesFile.toURI().toString();
-		edgesPath = edgesFile.toURI().toString();
-	}
-
-	@After
-	public void after() throws Exception{
-		compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01);
-	}
-
-	@Test
-	public void testPageRankSmallNumberOfIterations() throws Exception {
-		PageRank.main(new String[]{
-				"--pages", verticesPath,
-				"--links", edgesPath,
-				"--output", resultPath,
-				"--numPages", PageRankData.NUM_VERTICES + "",
-				"--iterations", "3"});
-		expected =  PageRankData.RANKS_AFTER_3_ITERATIONS;
-	}
-
-	@Test
-	public void testPageRankWithConvergenceCriterion() throws Exception {
-		PageRank.main(new String[]{
-				"--pages", verticesPath,
-				"--links", edgesPath,
-				"--output", resultPath,
-				"--numPages", PageRankData.NUM_VERTICES + "",
-				"--vertices", "1000"});
-		expected = PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.java
deleted file mode 100644
index 123f616..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/TransitiveClosureITCase.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.exampleJavaPrograms;
-
-
-import java.io.BufferedReader;
-
-import org.apache.flink.examples.java.graph.TransitiveClosureNaive;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.testdata.TransitiveClosureData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class TransitiveClosureITCase extends JavaProgramTestBase {
-
-	private static final long SEED = 0xBADC0FFEEBEEFL;
-
-	private static final int NUM_VERTICES = 100;
-
-	private static final int NUM_EDGES = 500;
-
-	private String edgesPath;
-	private String resultPath;
-
-
-	@Override
-	protected void preSubmit() throws Exception {
-		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
-		resultPath = getTempFilePath("results");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		TransitiveClosureNaive.main(
-				"--edges", edgesPath,
-				"--output", resultPath,
-				"--iterations", "5");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		for (BufferedReader reader : getResultReader(resultPath)) {
-			TransitiveClosureData.checkOddEvenResult(reader);
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
deleted file mode 100644
index b5ce8e4..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WebLogAnalysisITCase.java
+++ /dev/null
@@ -1,53 +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.exampleJavaPrograms;
-
-
-import org.apache.flink.examples.java.relational.WebLogAnalysis;
-import org.apache.flink.test.testdata.WebLogAnalysisData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class WebLogAnalysisITCase extends JavaProgramTestBase {
-
-	private String docsPath;
-	private String ranksPath;
-	private String visitsPath;
-	private String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		docsPath = createTempFile("docs", WebLogAnalysisData.DOCS);
-		ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS);
-		visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath);
-	}
-	@Override
-	protected void testProgram() throws Exception {
-		WebLogAnalysis.main(new String[] {
-				"--documents", docsPath,
-				"--ranks", ranksPath,
-				"--visits", visitsPath,
-				"--output", resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java
deleted file mode 100644
index 3c47c5c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.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.exampleJavaPrograms;
-
-import org.apache.flink.examples.java.wordcount.WordCount;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class WordCountITCase extends JavaProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WordCount.main(new String[] {
-				"--input", textPath,
-				"--output", resultPath });
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountNestedPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountNestedPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountNestedPOJOITCase.java
deleted file mode 100644
index f1e75fe..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountNestedPOJOITCase.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.exampleJavaPrograms;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-import java.util.Date;
-
-@SuppressWarnings("serial")
-public class WordCountNestedPOJOITCase extends JavaProgramTestBase implements Serializable {
-	private static final long serialVersionUID = 1L;
-	protected String textPath;
-	protected String resultPath;
-
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-	
-	@Override
-	protected void testProgram() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<String> text = env.readTextFile(textPath);
-
-		DataSet<WC> counts = text
-				.flatMap(new Tokenizer())
-				.groupBy("complex.someTest")
-				.reduce(new ReduceFunction<WC>() {
-					private static final long serialVersionUID = 1L;
-					public WC reduce(WC value1, WC value2) {
-						return new WC(value1.complex.someTest, value1.count + value2.count);
-					}
-				});
-
-		counts.writeAsText(resultPath);
-
-		env.execute("WordCount with custom data types example");
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, WC> {
-
-		@Override
-		public void flatMap(String value, Collector<WC> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new WC(token, 1));
-				}
-			}
-		}
-	}
-
-	public static class WC { // is a pojo
-		public ComplexNestedClass complex; // is a pojo
-		public int count; // is a BasicType
-
-		public WC() {
-		}
-		public WC(String t, int c) {
-			this.count = c;
-			this.complex = new ComplexNestedClass();
-			this.complex.word = new Tuple3<Long, Long, String>(0L, 0L, "egal");
-			this.complex.date = new Date();
-			this.complex.someFloat = 0.0f;
-			this.complex.someNumber = 666;
-			this.complex.someTest = t;
-		}
-		@Override
-		public String toString() {
-			return this.complex.someTest+" "+count;
-		}
-	}
-	
-	public static class ComplexNestedClass { // pojo
-		public static int ignoreStaticField;
-		public transient int ignoreTransientField;
-		public Date date; // generic type
-		public Integer someNumber; // BasicType
-		public float someFloat; // BasicType
-		public Tuple3<Long, Long, String> word; //Tuple Type with three basic types
-		public String someTest;
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSimplePOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSimplePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSimplePOJOITCase.java
deleted file mode 100644
index 7d20597..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSimplePOJOITCase.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.exampleJavaPrograms;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-
-
-public class WordCountSimplePOJOITCase extends JavaProgramTestBase implements Serializable {
-	private static final long serialVersionUID = 1L;
-	protected String textPath;
-	protected String resultPath;
-
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-	
-	@Override
-	protected void testProgram() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> text = env.readTextFile(textPath);
-
-		DataSet<WC> counts = text
-				.flatMap(new Tokenizer())
-				.groupBy("word")
-				.reduce(new ReduceFunction<WC>() {
-					private static final long serialVersionUID = 1L;
-
-					public WC reduce(WC value1, WC value2) {
-						return new WC(value1.word, value1.count + value2.count);
-					}
-				});
-
-		counts.writeAsText(resultPath);
-
-		env.execute("WordCount with custom data types example");
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, WC> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<WC> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new WC(token, 1));
-				}
-			}
-		}
-	}
-
-	public static class WC {
-		public WC() {}
-		public WC(String w, int c) {
-			word = w;
-			count = c;
-		}
-		public String word;
-		public int count;
-		@Override
-		public String toString() {
-			return word + " " + count;
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java
deleted file mode 100644
index 05ffc88..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java
+++ /dev/null
@@ -1,152 +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.exampleJavaPrograms;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class WordCountSubclassInterfacePOJOITCase extends JavaProgramTestBase implements Serializable {
-	private static final long serialVersionUID = 1L;
-	protected String textPath;
-	protected String resultPath;
-
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<String> text = env.readTextFile(textPath);
-
-		DataSet<WCBase> counts = text
-				.flatMap(new Tokenizer())
-				.groupBy("word")
-				.reduce(new ReduceFunction<WCBase>() {
-					private static final long serialVersionUID = 1L;
-					public WCBase reduce(WCBase value1, WCBase value2) {
-						WC wc1 = (WC) value1;
-						WC wc2 = (WC) value2;
-						int c = wc1.secretCount.getCount() + wc2.secretCount.getCount();
-						wc1.secretCount.setCount(c);
-						return wc1;
-					}
-				})
-				.map(new MapFunction<WCBase, WCBase>() {
-					@Override
-					public WCBase map(WCBase value) throws Exception {
-						WC wc = (WC) value;
-						wc.count = wc.secretCount.getCount();
-						return wc;
-					}
-				});
-
-		counts.writeAsText(resultPath);
-
-		env.execute("WordCount with custom data types example");
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, WCBase> {
-
-		@Override
-		public void flatMap(String value, Collector<WCBase> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new WC(token, 1));
-				}
-			}
-		}
-	}
-
-	public static abstract class WCBase {
-		public String word;
-		public int count;
-
-		public WCBase(String w, int c) {
-			this.word = w;
-			this.count = c;
-		}
-		@Override
-		public String toString() {
-			return word+" "+count;
-		}
-	}
-
-	public static interface CrazyCounter {
-		public int getCount();
-		public void setCount(int c);
-	}
-
-	public static class CrazyCounterImpl implements CrazyCounter {
-		public int countz;
-
-		public CrazyCounterImpl() {
-		}
-
-		public CrazyCounterImpl(int c) {
-			this.countz = c;
-		}
-
-		@Override
-		public int getCount() {
-			return countz;
-		}
-
-		@Override
-		public void setCount(int c) {
-			this.countz = c;
-		}
-
-	}
-
-	public static class WC extends WCBase {
-		public CrazyCounter secretCount;
-
-		public WC() {
-			super(null, 0);
-		}
-
-		public WC(String w, int c) {
-			super(w, 0);
-			this.secretCount = new CrazyCounterImpl(c);
-		}
-
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java
deleted file mode 100644
index f74ee16..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java
+++ /dev/null
@@ -1,123 +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.exampleJavaPrograms;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.util.Collector;
-
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class WordCountSubclassPOJOITCase extends JavaProgramTestBase implements Serializable {
-	private static final long serialVersionUID = 1L;
-	protected String textPath;
-	protected String resultPath;
-
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-	
-	@Override
-	protected void testProgram() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<String> text = env.readTextFile(textPath);
-
-		DataSet<WCBase> counts = text
-				.flatMap(new Tokenizer())
-				.groupBy("word")
-				.reduce(new ReduceFunction<WCBase>() {
-					private static final long serialVersionUID = 1L;
-					public WCBase reduce(WCBase value1, WCBase value2) {
-						WC wc1 = (WC) value1;
-						WC wc2 = (WC) value2;
-						return new WC(value1.word, wc1.secretCount + wc2.secretCount);
-					}
-				})
-				.map(new MapFunction<WCBase, WCBase>() {
-					@Override
-					public WCBase map(WCBase value) throws Exception {
-						WC wc = (WC) value;
-						wc.count = wc.secretCount;
-						return wc;
-					}
-				});
-
-		counts.writeAsText(resultPath);
-
-		env.execute("WordCount with custom data types example");
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, WCBase> {
-
-		@Override
-		public void flatMap(String value, Collector<WCBase> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new WC(token, 1));
-				}
-			}
-		}
-	}
-
-	public static abstract class WCBase {
-		public String word;
-		public int count;
-
-		public WCBase(String w, int c) {
-			this.word = w;
-			this.count = c;
-		}
-		@Override
-		public String toString() {
-			return word+" "+count;
-		}
-	}
-
-	public static class WC extends WCBase {
-
-		public int secretCount;
-
-		public WC() {
-			super(null, 0);
-		}
-
-		public WC(String w, int c) {
-			super(w, 0);
-			this.secretCount = c;
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.java
deleted file mode 100644
index d4404da..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountWithCollectionITCase.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.exampleJavaPrograms;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.examples.java.wordcount.WordCount;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.junit.Assert;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-public class WordCountWithCollectionITCase extends JavaProgramTestBase {
-
-	private final List<Tuple2<String, Integer>> resultsCollected = new ArrayList<Tuple2<String, Integer>>();
-
-	@Override
-	protected void postSubmit() throws Exception {
-		String[] result = new String[resultsCollected.size()];
-		for (int i = 0; i < result.length; i++) {
-			result[i] = resultsCollected.get(i).toString();
-		}
-		Arrays.sort(result);
-
-		String[] expected = WordCountData.COUNTS_AS_TUPLES.split("\n");
-		Arrays.sort(expected);
-
-		Assert.assertEquals("Different number of lines in expected and obtained result.", expected.length, result.length);
-		Assert.assertArrayEquals(expected, result);
-	}
-
-
-	@Override
-	protected void testProgram() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> text = env.fromElements(WordCountData.TEXT);
-		DataSet<Tuple2<String, Integer>> words = text.flatMap(new WordCount.Tokenizer());
-		DataSet<Tuple2<String, Integer>> result = words.groupBy(0).aggregate(Aggregations.SUM, 1);
-
-		result.output(new LocalCollectionOutputFormat<Tuple2<String, Integer>>(resultsCollected));
-		env.execute("Word Count Collection");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.java
deleted file mode 100644
index f3a1d03..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/ConnectedComponentsITCase.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.exampleScalaPrograms;
-
-import org.apache.flink.examples.scala.graph.ConnectedComponents;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.BufferedReader;
-
-public class ConnectedComponentsITCase extends JavaProgramTestBase {
-	
-	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
-	private static final int NUM_VERTICES = 1000;
-	
-	private static final int NUM_EDGES = 10000;
-
-	
-	private String verticesPath;
-	private String edgesPath;
-	private String resultPath;
-	
-	
-	@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 void testProgram() throws Exception {
-		ConnectedComponents.main(new String[] {
-				"--vertices", verticesPath,
-				"--edges", edgesPath,
-				"--output", resultPath,
-				"--iterations", "100"});
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		for (BufferedReader reader : getResultReader(resultPath)) {
-			ConnectedComponentsData.checkOddEvenResult(reader);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleITCase.java
deleted file mode 100644
index 9d5ffdf..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/EnumTriangleITCase.java
+++ /dev/null
@@ -1,48 +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.exampleScalaPrograms;
-
-import org.apache.flink.examples.scala.graph.EnumTriangles;
-import org.apache.flink.test.testdata.EnumTriangleData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class EnumTriangleITCase extends JavaProgramTestBase {
-	
-	protected String edgePath;
-	protected String resultPath;
-	
-	@Override
-	protected void preSubmit() throws Exception {
-		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
-		resultPath = getTempDirPath("triangles");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath);
-	}
-	
-	@Override
-	protected void testProgram() throws Exception {
-		EnumTriangles.main(new String[] {
-				"--edges", edgePath,
-				"--output", resultPath });
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
deleted file mode 100644
index e3e8936..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/PageRankITCase.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.exampleScalaPrograms;
-
-import java.io.File;
-
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-import org.apache.flink.examples.scala.graph.PageRankBasic;
-import org.apache.flink.test.testdata.PageRankData;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class PageRankITCase extends MultipleProgramsTestBase {
-
-	public PageRankITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	private String verticesPath;
-	private String edgesPath;
-	private String resultPath;
-	private String expected;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception{
-		File resultFile = tempFolder.newFile();
-		//Delete file because the Scala API does not respect WriteMode set by the configuration
-		resultFile.delete();
-		resultPath = resultFile.toURI().toString();
-
-		File verticesFile = tempFolder.newFile();
-		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
-
-		File edgesFile = tempFolder.newFile();
-		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
-
-		verticesPath = verticesFile.toURI().toString();
-		edgesPath = edgesFile.toURI().toString();
-	}
-
-	@After
-	public void after() throws Exception{
-		compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01);
-	}
-
-	@Test
-	public void testPageRankWithSmallNumberOfIterations() throws Exception {
-		PageRankBasic.main(new String[] {
-				"--pages", verticesPath,
-				"--links", edgesPath,
-				"--output", resultPath,
-				"--numPages", PageRankData.NUM_VERTICES+"",
-				"--iterations", "3"});
-		expected = PageRankData.RANKS_AFTER_3_ITERATIONS;
-	}
-
-	@Test
-	public void testPageRankWithConvergence() throws Exception {
-		// start with a very high number of iteration such that the dynamic convergence criterion must handle termination
-		PageRankBasic.main(new String[] {
-				"--pages", verticesPath,
-				"--links", edgesPath,
-				"--output", resultPath,
-				"--numPages", PageRankData.NUM_VERTICES+"",
-				"--iterations", "1000"});
-		expected = PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.java
deleted file mode 100644
index 9804f4c..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/TransitiveClosureITCase.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.exampleScalaPrograms;
-
-
-import java.io.BufferedReader;
-
-import org.apache.flink.examples.scala.graph.TransitiveClosureNaive;
-import org.apache.flink.test.testdata.ConnectedComponentsData;
-import org.apache.flink.test.testdata.TransitiveClosureData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class TransitiveClosureITCase extends JavaProgramTestBase {
-
-    private static final long SEED = 0xBADC0FFEEBEEFL;
-
-    private static final int NUM_VERTICES = 100;
-
-    private static final int NUM_EDGES = 500;
-
-    private String edgesPath;
-    private String resultPath;
-
-
-    @Override
-    protected void preSubmit() throws Exception {
-        edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
-        resultPath = getTempFilePath("results");
-    }
-
-    @Override
-    protected void testProgram() throws Exception {
-        TransitiveClosureNaive.main(new String [] {
-                "--edges", edgesPath,
-                "--output", resultPath,
-                "--iterations", "5"});
-    }
-
-    @Override
-    protected void postSubmit() throws Exception {
-        for (BufferedReader reader : getResultReader(resultPath)) {
-            TransitiveClosureData.checkOddEvenResult(reader);
-        }
-    }
-}
-


[20/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
index 4a1d181..7dd1144 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.checkpointing.utils;
 
-import static org.junit.Assert.assertEquals;
-
 import org.apache.flink.api.common.accumulators.IntCounter;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -53,9 +51,12 @@ import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.Collector;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+
 /**
  * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
  *
@@ -147,7 +148,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
 	}
 
-
 	@Test
 	public void testSavepointRestoreFromFlink12() throws Exception {
 
@@ -249,7 +249,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 	private static class LegacyCheckpointedSource
 			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
 
-		public static String CHECKPOINTED_STRING = "Here be dragons!";
+		public static String checkpointedString = "Here be dragons!";
 
 		private static final long serialVersionUID = 1L;
 
@@ -286,12 +286,12 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		@Override
 		public void restoreState(String state) throws Exception {
-			assertEquals(CHECKPOINTED_STRING, state);
+			assertEquals(checkpointedString, state);
 		}
 
 		@Override
 		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_STRING;
+			return checkpointedString;
 		}
 	}
 
@@ -322,7 +322,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		@Override
 		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
+			assertEquals(LegacyCheckpointedSource.checkpointedString, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 
 			// immediately trigger any set timers
@@ -350,12 +350,12 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements Checkpointed<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+		public static Tuple2<String, Long> checkpointedTuple =
 				new Tuple2<>("hello", 42L);
 
 		@Override
@@ -369,11 +369,11 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		@Override
 		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
+			return checkpointedTuple;
 		}
 	}
 
-	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements CheckpointedRestoring<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
@@ -393,7 +393,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
 			out.collect(value);
 
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 
 		}
@@ -404,13 +404,13 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class LegacyCheckpointedFlatMapWithKeyedState
+	private static class LegacyCheckpointedFlatMapWithKeyedState
 			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements Checkpointed<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+		public static Tuple2<String, Long> checkpointedTuple =
 				new Tuple2<>("hello", 42L);
 
 		private final ValueStateDescriptor<Long> stateDescriptor =
@@ -431,11 +431,11 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		@Override
 		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
+			return checkpointedTuple;
 		}
 	}
 
-	public static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements CheckpointedRestoring<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
@@ -464,7 +464,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 			}
 
 			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 		}
 
@@ -474,7 +474,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements CheckpointedRestoring<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
@@ -503,7 +503,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 			}
 
 			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 		}
 
@@ -513,7 +513,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -528,7 +528,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckingKeyedStateFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class CheckingKeyedStateFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -558,7 +558,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckpointedUdfOperator
+	private static class CheckpointedUdfOperator
 			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
@@ -591,7 +591,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckingRestoringUdfOperator
+	private static class CheckingRestoringUdfOperator
 			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
@@ -635,7 +635,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class TimelyStatefulOperator
+	private static class TimelyStatefulOperator
 			extends AbstractStreamOperator<Tuple2<Long, Long>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
 		private static final long serialVersionUID = 1L;
@@ -687,7 +687,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckingTimelyStatefulOperator
+	private static class CheckingTimelyStatefulOperator
 			extends AbstractStreamOperator<Tuple2<Long, Long>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
 		private static final long serialVersionUID = 1L;
@@ -751,7 +751,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
+	private static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
 		private static final long serialVersionUID = 1L;
 
 		public static final String NUM_ELEMENTS_ACCUMULATOR = AccumulatorCountingSink.class + "_NUM_ELEMENTS";

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index 8546368..31b6bcc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -41,17 +41,13 @@ import org.apache.flink.test.testdata.KMeansData;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
@@ -59,9 +55,18 @@ import java.net.URL;
 import java.util.Collections;
 import java.util.concurrent.TimeUnit;
 
+import scala.Option;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
+/**
+ * Test job classloader.
+ */
 public class ClassLoaderITCase extends TestLogger {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ClassLoaderITCase.class);
@@ -318,7 +323,7 @@ public class ClassLoaderITCase extends TestLogger {
 		// Trigger savepoint
 		String savepointPath = null;
 		for (int i = 0; i < 20; i++) {
-			LOG.info("Triggering savepoint (" + (i+1) + "/20).");
+			LOG.info("Triggering savepoint (" + (i + 1) + "/20).");
 			Future<Object> savepointFuture = jm.ask(new TriggerSavepoint(jobId, Option.<String>empty()), deadline.timeLeft());
 
 			Object savepointResponse = Await.result(savepointFuture, deadline.timeLeft());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
index 795ae41..51fad6b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointedStreamingProgram.java
@@ -27,7 +27,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 
-import java.lang.RuntimeException;
 import java.util.Collections;
 import java.util.List;
 
@@ -39,7 +38,7 @@ import java.util.List;
 public class CheckpointedStreamingProgram {
 
 	private static final int CHECKPOINT_INTERVALL = 100;
-	
+
 	public static void main(String[] args) throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -47,21 +46,20 @@ public class CheckpointedStreamingProgram {
 		env.enableCheckpointing(CHECKPOINT_INTERVALL);
 		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 100L));
 		env.disableOperatorChaining();
-		
+
 		DataStream<String> text = env.addSource(new SimpleStringGenerator());
 		text.map(new StatefulMapper()).addSink(new NoOpSink());
 		env.setParallelism(1);
 		env.execute("Checkpointed Streaming Program");
 	}
 
-
 	// with Checkpoining
-	public static class SimpleStringGenerator implements SourceFunction<String>, ListCheckpointed<Integer> {
+	private static class SimpleStringGenerator implements SourceFunction<String>, ListCheckpointed<Integer> {
 		public boolean running = true;
 
 		@Override
 		public void run(SourceContext<String> ctx) throws Exception {
-			while(running) {
+			while (running) {
 				Thread.sleep(1);
 				ctx.collect("someString");
 			}
@@ -83,7 +81,7 @@ public class CheckpointedStreamingProgram {
 		}
 	}
 
-	public static class StatefulMapper implements MapFunction<String, String>, ListCheckpointed<StatefulMapper>, CheckpointListener {
+	private static class StatefulMapper implements MapFunction<String, String>, ListCheckpointed<StatefulMapper>, CheckpointListener {
 
 		private String someState;
 		private boolean atLeastOneSnapshotComplete = false;
@@ -107,14 +105,14 @@ public class CheckpointedStreamingProgram {
 
 		@Override
 		public String map(String value) throws Exception {
-			if(!atLeastOneSnapshotComplete) {
+			if (!atLeastOneSnapshotComplete) {
 				// throttle consumption by the checkpoint interval until we have one snapshot.
 				Thread.sleep(CHECKPOINT_INTERVALL);
 			}
-			if(atLeastOneSnapshotComplete && !restored) {
+			if (atLeastOneSnapshotComplete && !restored) {
 				throw new RuntimeException("Intended failure, to trigger restore");
 			}
-			if(restored) {
+			if (restored) {
 				throw new SuccessException();
 				//throw new RuntimeException("All good");
 			}
@@ -130,13 +128,13 @@ public class CheckpointedStreamingProgram {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * We intentionally use a user specified failure exception
+	 * We intentionally use a user specified failure exception.
 	 */
-	public static class SuccessException extends Exception {
+	private static class SuccessException extends Exception {
 
 	}
 
-	public static class NoOpSink implements SinkFunction<String>{
+	private static class NoOpSink implements SinkFunction<String>{
 		@Override
 		public void invoke(String value) throws Exception {
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
index 2693bc1..aa8e59e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java
@@ -46,6 +46,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
 
+/**
+ * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}.
+ */
 public class CheckpointingCustomKvStateProgram {
 
 	public static void main(String[] args) throws Exception {
@@ -71,7 +74,7 @@ public class CheckpointingCustomKvStateProgram {
 						return new Tuple2<>(ThreadLocalRandom.current().nextInt(parallelism), value);
 					}
 				})
-				.keyBy(new KeySelector<Tuple2<Integer,Integer>, Integer>() {
+				.keyBy(new KeySelector<Tuple2<Integer, Integer>, Integer>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
@@ -133,12 +136,11 @@ public class CheckpointingCustomKvStateProgram {
 			this.kvState = getRuntimeContext().getReducingState(stateDescriptor);
 		}
 
-
 		@Override
 		public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
 			kvState.add(value.f1);
 
-			if(atLeastOneSnapshotComplete) {
+			if (atLeastOneSnapshotComplete) {
 				if (restored) {
 					throw new SuccessException();
 				} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
index 2caa7cf..a5a2531 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomInputSplitProgram.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.test.classloading.jar;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
@@ -36,9 +32,16 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}.
+ */
 @SuppressWarnings("serial")
 public class CustomInputSplitProgram {
-	
+
 	public static void main(String[] args) throws Exception {
 
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -58,8 +61,8 @@ public class CustomInputSplitProgram {
 		env.execute();
 	}
 	// --------------------------------------------------------------------------------------------
-	
-	public static final class CustomInputFormat implements InputFormat<Integer, CustomInputSplit>, ResultTypeQueryable<Integer> {
+
+	private static final class CustomInputFormat implements InputFormat<Integer, CustomInputSplit>, ResultTypeQueryable<Integer> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -113,7 +116,7 @@ public class CustomInputSplitProgram {
 		}
 	}
 
-	public static final class CustomInputSplit implements InputSplit {
+	private static final class CustomInputSplit implements InputSplit {
 
 		private static final long serialVersionUID = 1L;
 
@@ -129,7 +132,7 @@ public class CustomInputSplitProgram {
 		}
 	}
 
-	public static final class CustomSplitAssigner implements InputSplitAssigner {
+	private static final class CustomSplitAssigner implements InputSplitAssigner {
 
 		private final List<CustomInputSplit> remainingSplits;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
index cbd553c..819ad29 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java
@@ -64,7 +64,7 @@ public class CustomKvStateProgram {
 						return new Tuple2<>(ThreadLocalRandom.current().nextInt(parallelism), value);
 					}
 				})
-				.keyBy(new KeySelector<Tuple2<Integer,Integer>, Integer>() {
+				.keyBy(new KeySelector<Tuple2<Integer, Integer>, Integer>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
@@ -112,7 +112,6 @@ public class CustomKvStateProgram {
 			this.kvState = getRuntimeContext().getReducingState(stateDescriptor);
 		}
 
-
 		@Override
 		public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
 			kvState.add(value.f1);

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
index b8e6c85..72940c4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java
@@ -33,7 +33,7 @@ import java.util.Collection;
 /**
  * This class belongs to the {@link org.apache.flink.test.classloading.ClassLoaderITCase} test.
  *
- * It tests dynamic class loading for:
+ * <p>It tests dynamic class loading for:
  * <ul>
  *     <li>Custom Functions</li>
  *     <li>Custom Data Types</li>
@@ -41,8 +41,7 @@ import java.util.Collection;
  *     <li>Custom Types in collect()</li>
  * </ul>
  *
- * <p>
- * It's removed by Maven from classpath, so other tests must not depend on it.
+ * <p>It's removed by Maven from classpath, so other tests must not depend on it.
  */
 @SuppressWarnings("serial")
 public class KMeansForTest {
@@ -124,7 +123,7 @@ public class KMeansForTest {
 		}
 
 		public double euclideanDistance(Point other) {
-			return Math.sqrt((x-other.x)*(x-other.x) + (y-other.y)*(y-other.y));
+			return Math.sqrt((x - other.x) * (x - other.x) + (y - other.y) * (y - other.y));
 		}
 
 		public void clear() {
@@ -147,7 +146,7 @@ public class KMeansForTest {
 		public Centroid() {}
 
 		public Centroid(int id, double x, double y) {
-			super(x,y);
+			super(x, y);
 			this.id = id;
 		}
 
@@ -166,7 +165,9 @@ public class KMeansForTest {
 	//     USER FUNCTIONS
 	// *************************************************************************
 
-	/** Converts a Tuple2<Double,Double> into a Point. */
+	/**
+	 * Converts a {@code Tuple2<Double, Double>} into a {@link Point}.
+	 */
 	public static final class TuplePointConverter extends RichMapFunction<String, Point> {
 
 		@Override
@@ -176,7 +177,9 @@ public class KMeansForTest {
 		}
 	}
 
-	/** Converts a Tuple3<Integer, Double,Double> into a Centroid. */
+	/**
+	 * Converts a {@code Tuple3<Integer, Double, Double>} into a {@link Centroid}.
+	 */
 	public static final class TupleCentroidConverter extends RichMapFunction<String, Centroid> {
 
 		@Override
@@ -186,7 +189,9 @@ public class KMeansForTest {
 		}
 	}
 
-	/** Determines the closest cluster center for a data point. */
+	/**
+	 * Determines the closest cluster center for a data point.
+	 */
 	public static final class SelectNearestCenter extends RichMapFunction<Point, Tuple2<Integer, Point>> {
 
 		private Collection<Centroid> centroids;
@@ -197,7 +202,7 @@ public class KMeansForTest {
 		public void open(Configuration parameters) throws Exception {
 			this.centroids = getRuntimeContext().getBroadcastVariable("centroids");
 			this.acc = new CustomAccumulator();
-			 getRuntimeContext().addAccumulator("myAcc", this.acc);
+			getRuntimeContext().addAccumulator("myAcc", this.acc);
 		}
 
 		@Override
@@ -224,7 +229,9 @@ public class KMeansForTest {
 		}
 	}
 
-	// Use this so that we can check whether POJOs and the POJO comparator also work
+	/**
+	 * 	Use this so that we can check whether POJOs and the POJO comparator also work.
+	 */
 	public static final class DummyTuple3IntPointLong {
 		public Integer field0;
 		public Point field1;
@@ -239,7 +246,9 @@ public class KMeansForTest {
 		}
 	}
 
-	/** Appends a count variable to the tuple. */
+	/**
+	 * Appends a count variable to the tuple.
+	 */
 	public static final class CountAppender extends RichMapFunction<Tuple2<Integer, Point>, DummyTuple3IntPointLong> {
 
 		@Override
@@ -248,7 +257,9 @@ public class KMeansForTest {
 		}
 	}
 
-	/** Sums and counts point coordinates. */
+	/**
+	 * Sums and counts point coordinates.
+	 */
 	public static final class CentroidAccumulator extends RichReduceFunction<DummyTuple3IntPointLong> {
 
 		@Override
@@ -257,7 +268,9 @@ public class KMeansForTest {
 		}
 	}
 
-	/** Computes new centroid from coordinate sum and count of points. */
+	/**
+	 * Computes new centroid from coordinate sum and count of points.
+	 */
 	public static final class CentroidAverager extends RichMapFunction<DummyTuple3IntPointLong, Centroid> {
 
 		@Override
@@ -266,7 +279,7 @@ public class KMeansForTest {
 		}
 	}
 
-	public static class CustomAccumulator implements SimpleAccumulator<Long> {
+	private static class CustomAccumulator implements SimpleAccumulator<Long> {
 
 		private long value;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
index 210973f..1431d96 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
@@ -50,9 +50,8 @@ public class LegacyCheckpointedStreamingProgram {
 		env.execute("Checkpointed Streaming Program");
 	}
 
-
 	// with Checkpointing
-	public static class SimpleStringGenerator implements SourceFunction<String>, Checkpointed<Integer> {
+	private static class SimpleStringGenerator implements SourceFunction<String>, Checkpointed<Integer> {
 
 		private static final long serialVersionUID = 3700033137820808611L;
 
@@ -60,7 +59,7 @@ public class LegacyCheckpointedStreamingProgram {
 
 		@Override
 		public void run(SourceContext<String> ctx) throws Exception {
-			while(running) {
+			while (running) {
 				Thread.sleep(1);
 				ctx.collect("someString");
 			}
@@ -82,7 +81,7 @@ public class LegacyCheckpointedStreamingProgram {
 		}
 	}
 
-	public static class StatefulMapper implements MapFunction<String, String>, Checkpointed<StatefulMapper>, CheckpointListener {
+	private static class StatefulMapper implements MapFunction<String, String>, Checkpointed<StatefulMapper>, CheckpointListener {
 
 		private static final long serialVersionUID = 2703630582894634440L;
 
@@ -104,14 +103,14 @@ public class LegacyCheckpointedStreamingProgram {
 
 		@Override
 		public String map(String value) throws Exception {
-			if(!atLeastOneSnapshotComplete) {
+			if (!atLeastOneSnapshotComplete) {
 				// throttle consumption by the checkpoint interval until we have one snapshot.
 				Thread.sleep(CHECKPOINT_INTERVALL);
 			}
-			if(atLeastOneSnapshotComplete && !restored) {
+			if (atLeastOneSnapshotComplete && !restored) {
 				throw new RuntimeException("Intended failure, to trigger restore");
 			}
-			if(restored) {
+			if (restored) {
 				throw new SuccessException();
 				//throw new RuntimeException("All good");
 			}
@@ -127,14 +126,14 @@ public class LegacyCheckpointedStreamingProgram {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * We intentionally use a user specified failure exception
+	 * We intentionally use a user specified failure exception.
 	 */
-	public static class SuccessException extends Exception {
+	private static class SuccessException extends Exception {
 
 		private static final long serialVersionUID = 7073311460437532086L;
 	}
 
-	public static class NoOpSink implements SinkFunction<String> {
+	private static class NoOpSink implements SinkFunction<String> {
 		private static final long serialVersionUID = 2381410324190818620L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
index 4905d43..69421a6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingCustomInputSplitProgram.java
@@ -38,9 +38,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+/**
+ * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}.
+ */
 @SuppressWarnings("serial")
 public class StreamingCustomInputSplitProgram {
-	
+
 	public static void main(String[] args) throws Exception {
 				Configuration config = new Configuration();
 
@@ -62,8 +65,8 @@ public class StreamingCustomInputSplitProgram {
 		env.execute();
 	}
 	// --------------------------------------------------------------------------------------------
-	
-	public static final class CustomInputFormat implements InputFormat<Integer, CustomInputSplit>, ResultTypeQueryable<Integer> {
+
+	private static final class CustomInputFormat implements InputFormat<Integer, CustomInputSplit>, ResultTypeQueryable<Integer> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -119,7 +122,7 @@ public class StreamingCustomInputSplitProgram {
 		}
 	}
 
-	public static final class CustomInputSplit implements InputSplit {
+	private static final class CustomInputSplit implements InputSplit {
 
 		private static final long serialVersionUID = 1L;
 
@@ -135,7 +138,7 @@ public class StreamingCustomInputSplitProgram {
 		}
 	}
 
-	public static final class CustomSplitAssigner implements InputSplitAssigner, Serializable {
+	private static final class CustomSplitAssigner implements InputSplitAssigner, Serializable {
 
 		private final List<CustomInputSplit> remainingSplits;
 
@@ -156,7 +159,7 @@ public class StreamingCustomInputSplitProgram {
 		}
 	}
 
-	public static class NoOpSink implements SinkFunction<Tuple2<Integer, Double>> {
+	private static class NoOpSink implements SinkFunction<Tuple2<Integer, Double>> {
 		@Override
 		public void invoke(Tuple2<Integer, Double> value) throws Exception {
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
index 0fdc744..596e4dd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/StreamingProgram.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.classloading.jar;
 
-import java.util.StringTokenizer;
-
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -27,13 +25,18 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.util.Collector;
 
+import java.util.StringTokenizer;
+
+/**
+ * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}.
+ */
 @SuppressWarnings("serial")
 public class StreamingProgram {
-	
+
 	public static void main(String[] args) throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
-		
+
 		DataStream<String> text = env.fromElements(WordCountData.TEXT).rebalance();
 
 		DataStream<Word> counts =
@@ -45,6 +48,9 @@ public class StreamingProgram {
 	}
 	// --------------------------------------------------------------------------------------------
 
+	/**
+	 * POJO with word and count.
+	 */
 	public static class Word {
 
 		private String word;
@@ -80,7 +86,7 @@ public class StreamingProgram {
 		}
 	}
 
-	public static class Tokenizer implements FlatMapFunction<String, Word>{
+	private static class Tokenizer implements FlatMapFunction<String, Word>{
 		@Override
 		public void flatMap(String value, Collector<Word> out) throws Exception {
 			StringTokenizer tokenizer = new StringTokenizer(value);
@@ -90,7 +96,7 @@ public class StreamingProgram {
 		}
 	}
 
-	public static class NoOpSink implements SinkFunction<Word>{
+	private static class NoOpSink implements SinkFunction<Word>{
 		@Override
 		public void invoke(Word value) throws Exception {
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
index f12fd5f..a15a7a4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/UserCodeType.java
@@ -26,10 +26,10 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 /**
  * Test class used by the {@link org.apache.flink.test.classloading.ClassLoaderITCase}.
  *
- * This class is used to test FLINK-3633
+ * <p>This class is used to test FLINK-3633
  */
 public class UserCodeType {
-	public static class CustomType {
+	private static class CustomType {
 		private final int value;
 
 		public CustomType(int value) {
@@ -46,7 +46,7 @@ public class UserCodeType {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.getConfig().disableSysoutLogging();
 
-		DataSet<Integer> input = env.fromElements(1,2,3,4,5);
+		DataSet<Integer> input = env.fromElements(1, 2, 3, 4, 5);
 
 		DataSet<CustomType> customTypes = input.map(new MapFunction<Integer, CustomType>() {
 			private static final long serialVersionUID = -5878758010124912128L;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
deleted file mode 100644
index 61595f2..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
+++ /dev/null
@@ -1,140 +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.clients.examples;
-
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.ClusterClient;
-import org.apache.flink.client.program.StandaloneClusterClient;
-import org.apache.flink.runtime.client.JobRetrievalException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
-import org.apache.flink.runtime.testingUtils.TestingCluster;
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import scala.collection.Seq;
-
-import java.util.concurrent.Semaphore;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-/**
- * Tests retrieval of a job from a running Flink cluster
- */
-public class JobRetrievalITCase extends TestLogger {
-
-	private static final Semaphore lock = new Semaphore(1);
-
-	private static FlinkMiniCluster cluster;
-
-	@BeforeClass
-	public static void before() {
-		Configuration configuration = new Configuration();
-		cluster = new TestingCluster(configuration, false);
-		cluster.start();
-	}
-
-	@AfterClass
-	public static void after() {
-		cluster.stop();
-		cluster = null;
-	}
-
-	@Test
-	public void testJobRetrieval() throws Exception {
-		final JobID jobID = new JobID();
-
-		final JobVertex imalock = new JobVertex("imalock");
-		imalock.setInvokableClass(SemaphoreInvokable.class);
-
-		final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock);
-
-		final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
-
-		// acquire the lock to make sure that the job cannot complete until the job client
-		// has been attached in resumingThread
-		lock.acquire();
-		client.runDetached(jobGraph, JobRetrievalITCase.class.getClassLoader());
-
-		final Thread resumingThread = new Thread(new Runnable() {
-			@Override
-			public void run() {
-				try {
-					assertNotNull(client.retrieveJob(jobID));
-				} catch (Throwable e) {
-					fail(e.getMessage());
-				}
-			}
-		});
-
-		final Seq<ActorSystem> actorSystemSeq = cluster.jobManagerActorSystems().get();
-		final ActorSystem actorSystem = actorSystemSeq.last();
-		JavaTestKit testkit = new JavaTestKit(actorSystem);
-
-		final ActorRef jm = cluster.getJobManagersAsJava().get(0);
-		// wait until client connects
-		jm.tell(TestingJobManagerMessages.getNotifyWhenClientConnects(), testkit.getRef());
-		// confirm registration
-		testkit.expectMsgEquals(true);
-
-		// kick off resuming
-		resumingThread.start();
-
-		// wait for client to connect
-		testkit.expectMsgAllOf(
-			TestingJobManagerMessages.getClientConnected(),
-			TestingJobManagerMessages.getClassLoadingPropsDelivered());
-
-		// client has connected, we can release the lock
-		lock.release();
-
-		resumingThread.join();
-	}
-
-	@Test
-	public void testNonExistingJobRetrieval() throws Exception {
-		final JobID jobID = new JobID();
-		ClusterClient client = new StandaloneClusterClient(cluster.configuration());
-
-		try {
-			client.retrieveJob(jobID);
-			fail();
-		} catch (JobRetrievalException ignored) {
-			// this is what we want
-		}
-	}
-
-
-	public static class SemaphoreInvokable extends AbstractInvokable {
-
-		@Override
-		public void invoke() throws Exception {
-			lock.acquire();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.java
deleted file mode 100644
index 606cdc1..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/LocalExecutorITCase.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.clients.examples;
-
-import java.io.File;
-import java.io.FileWriter;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.client.LocalExecutor;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.testfunctions.Tokenizer;
-import org.apache.flink.util.TestLogger;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class LocalExecutorITCase extends TestLogger {
-
-	private static final int parallelism = 4;
-
-	@Test
-	public void testLocalExecutorWithWordCount() {
-		try {
-			// set up the files
-			File inFile = File.createTempFile("wctext", ".in");
-			File outFile = File.createTempFile("wctext", ".out");
-			inFile.deleteOnExit();
-			outFile.deleteOnExit();
-
-			try (FileWriter fw = new FileWriter(inFile)) {
-				fw.write(WordCountData.TEXT);
-			}
-
-			LocalExecutor executor = new LocalExecutor();
-			executor.setDefaultOverwriteFiles(true);
-			executor.setTaskManagerNumSlots(parallelism);
-			executor.setPrintStatusDuringExecution(false);
-			executor.start();
-			Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism);
-			wcPlan.setExecutionConfig(new ExecutionConfig());
-			executor.executePlan(wcPlan);
-			executor.stop();
-		} catch (Exception e) {
-			e.printStackTrace();
-			Assert.fail(e.getMessage());
-		}
-	}
-
-	private Plan getWordCountPlan(File inFile, File outFile, int parallelism) {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(parallelism);
-		env.readTextFile(inFile.getAbsolutePath())
-			.flatMap(new Tokenizer())
-			.groupBy(0)
-			.sum(1)
-			.writeAsCsv(outFile.getAbsolutePath());
-		return env.createProgramPlan();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
index b4549a8..91566af 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/completeness/TypeInfoTestCoverageTest.java
@@ -18,15 +18,18 @@
 
 package org.apache.flink.test.completeness;
 
-import java.lang.reflect.Modifier;
-import java.util.Set;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.apache.flink.util.TestLogger;
-import static org.junit.Assert.assertTrue;
+
 import org.junit.Test;
 import org.reflections.Reflections;
 
+import java.lang.reflect.Modifier;
+import java.util.Set;
+
+import static org.junit.Assert.assertTrue;
+
 /**
  * Scans the class path for type information and checks if there is a test for it.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java b/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java
deleted file mode 100644
index 21aa40a..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/distributedCache/DistributedCacheTest.java
+++ /dev/null
@@ -1,123 +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.distributedCache;
-
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.test.util.TestEnvironment;
-import org.apache.flink.util.Collector;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
-
-public class DistributedCacheTest extends AbstractTestBase {
-
-	public static final String data
-			= "machen\n"
-			+ "zeit\n"
-			+ "heerscharen\n"
-			+ "keiner\n"
-			+ "meine\n";
-
-	private static final int PARALLELISM = 4;
-
-	private static LocalFlinkMiniCluster cluster;
-
-	@BeforeClass
-	public static void setup() throws Exception {
-		cluster = TestBaseUtils.startCluster(1, PARALLELISM, false, false, true);
-		TestStreamEnvironment.setAsContext(cluster, PARALLELISM);
-		TestEnvironment.setAsContext(cluster, PARALLELISM);
-	}
-
-	@AfterClass
-	public static void teardown() throws Exception {
-		TestStreamEnvironment.unsetAsContext();
-		TestEnvironment.unsetAsContext();
-		TestBaseUtils.stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
-	}
-
-	// ------------------------------------------------------------------------
-
-	public DistributedCacheTest() {
-		super(new Configuration());
-	}
-	
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testStreamingDistributedCache() throws Exception {
-		String textPath = createTempFile("count.txt", data);
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.registerCachedFile(textPath, "cache_test");
-		env.readTextFile(textPath).flatMap(new WordChecker());
-		env.execute();
-	}
-
-	@Test
-	public void testBatchDistributedCache() throws Exception {
-		String textPath = createTempFile("count.txt", data);
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.registerCachedFile(textPath, "cache_test");
-		env.readTextFile(textPath).flatMap(new WordChecker()).count();
-	}
-
-	public static class WordChecker extends RichFlatMapFunction<String, Tuple1<String>> {
-		private static final long serialVersionUID = 1L;
-
-		private final List<String> wordList = new ArrayList<>();
-
-		@Override
-		public void open(Configuration conf) throws IOException {
-			File file = getRuntimeContext().getDistributedCache().getFile("cache_test");
-			try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
-				String tempString;
-				while ((tempString= reader.readLine()) != null) {
-					wordList.add(tempString);
-				}
-			}
-		}
-
-		@Override
-		public void flatMap(String word, Collector<Tuple1<String>> out) throws Exception {
-			assertTrue("Unexpected word in stream! wordFromStream: " + word + ", shouldBeOneOf: " +
-				wordList.toString(), wordList.contains(word));
-
-			out.collect(new Tuple1<>(word));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java
new file mode 100644
index 0000000..63ce3ab
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.distributedcache;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.test.util.TestEnvironment;
+import org.apache.flink.util.Collector;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test the distributed cache.
+ */
+public class DistributedCacheTest extends AbstractTestBase {
+
+	public static final String DATA =
+			"machen\n" +
+			"zeit\n" +
+			"heerscharen\n" +
+			"keiner\n" +
+			"meine\n";
+
+	private static final int PARALLELISM = 4;
+
+	private static LocalFlinkMiniCluster cluster;
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		cluster = TestBaseUtils.startCluster(1, PARALLELISM, false, false, true);
+		TestStreamEnvironment.setAsContext(cluster, PARALLELISM);
+		TestEnvironment.setAsContext(cluster, PARALLELISM);
+	}
+
+	@AfterClass
+	public static void teardown() throws Exception {
+		TestStreamEnvironment.unsetAsContext();
+		TestEnvironment.unsetAsContext();
+		TestBaseUtils.stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
+	}
+
+	// ------------------------------------------------------------------------
+
+	public DistributedCacheTest() {
+		super(new Configuration());
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testStreamingDistributedCache() throws Exception {
+		String textPath = createTempFile("count.txt", DATA);
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.registerCachedFile(textPath, "cache_test");
+		env.readTextFile(textPath).flatMap(new WordChecker());
+		env.execute();
+	}
+
+	@Test
+	public void testBatchDistributedCache() throws Exception {
+		String textPath = createTempFile("count.txt", DATA);
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.registerCachedFile(textPath, "cache_test");
+		env.readTextFile(textPath).flatMap(new WordChecker()).count();
+	}
+
+	private static class WordChecker extends RichFlatMapFunction<String, Tuple1<String>> {
+		private static final long serialVersionUID = 1L;
+
+		private final List<String> wordList = new ArrayList<>();
+
+		@Override
+		public void open(Configuration conf) throws IOException {
+			File file = getRuntimeContext().getDistributedCache().getFile("cache_test");
+			try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
+				String tempString;
+				while ((tempString = reader.readLine()) != null) {
+					wordList.add(tempString);
+				}
+			}
+		}
+
+		@Override
+		public void flatMap(String word, Collector<Tuple1<String>> out) throws Exception {
+			assertTrue("Unexpected word in stream! wordFromStream: " + word + ", shouldBeOneOf: " +
+				wordList.toString(), wordList.contains(word));
+
+			out.collect(new Tuple1<>(word));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
new file mode 100644
index 0000000..6ce4f76
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.StandaloneClusterClient;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobRetrievalException;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
+import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.concurrent.Semaphore;
+
+import scala.collection.Seq;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests retrieval of a job from a running Flink cluster.
+ */
+public class JobRetrievalITCase extends TestLogger {
+
+	private static final Semaphore lock = new Semaphore(1);
+
+	private static FlinkMiniCluster cluster;
+
+	@BeforeClass
+	public static void before() {
+		Configuration configuration = new Configuration();
+		cluster = new TestingCluster(configuration, false);
+		cluster.start();
+	}
+
+	@AfterClass
+	public static void after() {
+		cluster.stop();
+		cluster = null;
+	}
+
+	@Test
+	public void testJobRetrieval() throws Exception {
+		final JobID jobID = new JobID();
+
+		final JobVertex imalock = new JobVertex("imalock");
+		imalock.setInvokableClass(SemaphoreInvokable.class);
+
+		final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock);
+
+		final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices());
+
+		// acquire the lock to make sure that the job cannot complete until the job client
+		// has been attached in resumingThread
+		lock.acquire();
+		client.runDetached(jobGraph, JobRetrievalITCase.class.getClassLoader());
+
+		final Thread resumingThread = new Thread(new Runnable() {
+			@Override
+			public void run() {
+				try {
+					assertNotNull(client.retrieveJob(jobID));
+				} catch (Throwable e) {
+					fail(e.getMessage());
+				}
+			}
+		});
+
+		final Seq<ActorSystem> actorSystemSeq = cluster.jobManagerActorSystems().get();
+		final ActorSystem actorSystem = actorSystemSeq.last();
+		JavaTestKit testkit = new JavaTestKit(actorSystem);
+
+		final ActorRef jm = cluster.getJobManagersAsJava().get(0);
+		// wait until client connects
+		jm.tell(TestingJobManagerMessages.getNotifyWhenClientConnects(), testkit.getRef());
+		// confirm registration
+		testkit.expectMsgEquals(true);
+
+		// kick off resuming
+		resumingThread.start();
+
+		// wait for client to connect
+		testkit.expectMsgAllOf(
+			TestingJobManagerMessages.getClientConnected(),
+			TestingJobManagerMessages.getClassLoadingPropsDelivered());
+
+		// client has connected, we can release the lock
+		lock.release();
+
+		resumingThread.join();
+	}
+
+	@Test
+	public void testNonExistingJobRetrieval() throws Exception {
+		final JobID jobID = new JobID();
+		ClusterClient client = new StandaloneClusterClient(cluster.configuration());
+
+		try {
+			client.retrieveJob(jobID);
+			fail();
+		} catch (JobRetrievalException ignored) {
+			// this is what we want
+		}
+	}
+
+	private static class SemaphoreInvokable extends AbstractInvokable {
+
+		@Override
+		public void invoke() throws Exception {
+			lock.acquire();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java
new file mode 100644
index 0000000..204d2a8
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.client;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.client.LocalExecutor;
+import org.apache.flink.test.testdata.WordCountData;
+import org.apache.flink.test.testfunctions.Tokenizer;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+
+/**
+ * Integration tests for {@link LocalExecutor}.
+ */
+public class LocalExecutorITCase extends TestLogger {
+
+	private static final int parallelism = 4;
+
+	@Test
+	public void testLocalExecutorWithWordCount() {
+		try {
+			// set up the files
+			File inFile = File.createTempFile("wctext", ".in");
+			File outFile = File.createTempFile("wctext", ".out");
+			inFile.deleteOnExit();
+			outFile.deleteOnExit();
+
+			try (FileWriter fw = new FileWriter(inFile)) {
+				fw.write(WordCountData.TEXT);
+			}
+
+			LocalExecutor executor = new LocalExecutor();
+			executor.setDefaultOverwriteFiles(true);
+			executor.setTaskManagerNumSlots(parallelism);
+			executor.setPrintStatusDuringExecution(false);
+			executor.start();
+			Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism);
+			wcPlan.setExecutionConfig(new ExecutionConfig());
+			executor.executePlan(wcPlan);
+			executor.stop();
+		} catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+	}
+
+	private Plan getWordCountPlan(File inFile, File outFile, int parallelism) {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(parallelism);
+		env.readTextFile(inFile.getAbsolutePath())
+			.flatMap(new Tokenizer())
+			.groupBy(0)
+			.sum(1)
+			.writeAsCsv(outFile.getAbsolutePath());
+		return env.createProgramPlan();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
new file mode 100644
index 0000000..1383894
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.failing;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for failing job submissions.
+ */
+@RunWith(Parameterized.class)
+public class JobSubmissionFailsITCase extends TestLogger {
+
+	private static final int NUM_SLOTS = 20;
+
+	private static LocalFlinkMiniCluster cluster;
+	private static JobGraph workingJobGraph;
+
+	@BeforeClass
+	public static void setup() {
+		try {
+			Configuration config = new Configuration();
+			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
+			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2);
+
+			cluster = new LocalFlinkMiniCluster(config);
+
+			cluster.start();
+
+			final JobVertex jobVertex = new JobVertex("Working job vertex.");
+			jobVertex.setInvokableClass(NoOpInvokable.class);
+			workingJobGraph = new JobGraph("Working testing job", jobVertex);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@AfterClass
+	public static void teardown() {
+		try {
+			cluster.shutdown();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private boolean detached;
+
+	public JobSubmissionFailsITCase(boolean detached) {
+		this.detached = detached;
+	}
+
+	@Parameterized.Parameters(name = "Detached mode = {0}")
+	public static Collection<Boolean[]> executionModes(){
+		return Arrays.asList(new Boolean[]{false},
+				new Boolean[]{true});
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private JobExecutionResult submitJob(JobGraph jobGraph) throws Exception {
+		if (detached) {
+			cluster.submitJobDetached(jobGraph);
+			return null;
+		}
+		else {
+			return cluster.submitJobAndWait(jobGraph, false, TestingUtils.TESTING_DURATION());
+		}
+	}
+
+	@Test
+	public void testExceptionInInitializeOnMaster() {
+		try {
+			final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex");
+			failingJobVertex.setInvokableClass(NoOpInvokable.class);
+
+			final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex);
+
+			try {
+				submitJob(failingJobGraph);
+				fail("Expected JobExecutionException.");
+			}
+			catch (JobExecutionException e) {
+				assertEquals("Test exception.", e.getCause().getMessage());
+			}
+			catch (Throwable t) {
+				t.printStackTrace();
+				fail("Caught wrong exception of type " + t.getClass() + ".");
+			}
+
+			cluster.submitJobAndWait(workingJobGraph, false);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSubmitEmptyJobGraph() {
+		try {
+			final JobGraph jobGraph = new JobGraph("Testing job");
+
+			try {
+				submitJob(jobGraph);
+				fail("Expected JobSubmissionException.");
+			}
+			catch (JobSubmissionException e) {
+				assertTrue(e.getMessage() != null && e.getMessage().contains("empty"));
+			}
+			catch (Throwable t) {
+				t.printStackTrace();
+				fail("Caught wrong exception of type " + t.getClass() + ".");
+			}
+
+			cluster.submitJobAndWait(workingJobGraph, false);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSubmitNullJobGraph() {
+		try {
+			try {
+				submitJob(null);
+				fail("Expected JobSubmissionException.");
+			}
+			catch (NullPointerException e) {
+				// yo!
+			}
+			catch (Throwable t) {
+				t.printStackTrace();
+				fail("Caught wrong exception of type " + t.getClass() + ".");
+			}
+
+			cluster.submitJobAndWait(workingJobGraph, false);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private static class FailingJobVertex extends JobVertex {
+		private static final long serialVersionUID = -6365291240199412135L;
+
+		public FailingJobVertex(final String msg) {
+			super(msg);
+		}
+
+		@Override
+		public void initializeOnMaster(ClassLoader loader) throws Exception {
+			throw new Exception("Test exception.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java
new file mode 100644
index 0000000..b830508
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.failing;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Assert;
+
+import java.util.List;
+
+/**
+ * Tests that both jobs, the failing and the working one, are handled correctly. The first (failing) job must be
+ * canceled and the client must report the failure. The second (working) job must finish successfully and compute the
+ * correct result.
+ *
+ */
+public class TaskFailureITCase extends JavaProgramTestBase {
+
+	private static final String EXCEPTION_STRING = "This is an expected Test Exception";
+
+	@Override
+	protected void testProgram() throws Exception {
+		//test failing version
+		try {
+			executeTask(new FailingTestMapper(), 1);
+		} catch (RuntimeException e) { //expected for collection execution
+			if (!isCollectionExecution()) {
+				Assert.fail();
+			}
+			// for collection execution, no restarts. So, exception should be appended with 0.
+			Assert.assertEquals(EXCEPTION_STRING + ":0", e.getMessage());
+		} catch (JobExecutionException e) { //expected for cluster execution
+			if (isCollectionExecution()) {
+				Assert.fail();
+			}
+			// for cluster execution, one restart. So, exception should be appended with 1.
+			Assert.assertEquals(EXCEPTION_STRING + ":1", e.getCause().getMessage());
+		}
+		//test correct version
+		executeTask(new TestMapper(), 0);
+	}
+
+	private void executeTask(MapFunction<Long, Long> mapper, int retries) throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(retries, 0));
+		List<Long> result = env.generateSequence(1, 9)
+				.map(mapper)
+				.collect();
+		MultipleProgramsTestBase.compareResultAsText(result, "1\n2\n3\n4\n5\n6\n7\n8\n9");
+	}
+
+	/**
+	 * Working map function.
+	 */
+	public static class TestMapper implements MapFunction<Long, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long map(Long value) throws Exception {
+			return value;
+		}
+	}
+
+	/**
+	 * Failing map function.
+	 */
+	public static class FailingTestMapper extends RichMapFunction<Long, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long map(Long value) throws Exception {
+			throw new RuntimeException(EXCEPTION_STRING + ":" + getRuntimeContext().getAttemptNumber());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java
new file mode 100644
index 0000000..c3bfd5d
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/ConnectedComponentsITCase.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.examples.java.graph.ConnectedComponents;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.BufferedReader;
+
+/**
+ * Test for {@link ConnectedComponents}.
+ */
+public class ConnectedComponentsITCase extends JavaProgramTestBase {
+
+	private static final long SEED = 0xBADC0FFEEBEEFL;
+
+	private static final int NUM_VERTICES = 1000;
+
+	private static final int NUM_EDGES = 10000;
+
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+
+	@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 void testProgram() throws Exception {
+		ConnectedComponents.main(
+				"--vertices", verticesPath,
+				"--edges", edgesPath,
+				"--output", resultPath,
+				"--iterations", "100");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			ConnectedComponentsData.checkOddEvenResult(reader);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java
new file mode 100644
index 0000000..e176ce8
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/EnumTriangleBasicITCase.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.example.java;
+
+import org.apache.flink.examples.java.graph.EnumTriangles;
+import org.apache.flink.test.testdata.EnumTriangleData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test {@link EnumTriangles}.
+ */
+public class EnumTriangleBasicITCase extends JavaProgramTestBase {
+
+	protected String edgePath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		edgePath = createTempFile("edges", EnumTriangleData.EDGES);
+		resultPath = getTempDirPath("triangles");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(EnumTriangleData.TRIANGLES_BY_ID, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		EnumTriangles.main(new String[] {
+				"--edges", edgePath,
+				"--output", resultPath });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
new file mode 100644
index 0000000..daab163
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/PageRankITCase.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.examples.java.graph.PageRank;
+import org.apache.flink.test.testdata.PageRankData;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+
+/**
+ * Test for {@link PageRank}.
+ */
+@RunWith(Parameterized.class)
+public class PageRankITCase extends MultipleProgramsTestBase {
+
+	public PageRankITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	private String verticesPath;
+	private String edgesPath;
+	private String resultPath;
+	private String expected;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Before
+	public void before() throws Exception{
+		resultPath = tempFolder.newFile().toURI().toString();
+		File verticesFile = tempFolder.newFile();
+		Files.write(PageRankData.VERTICES, verticesFile, Charsets.UTF_8);
+
+		File edgesFile = tempFolder.newFile();
+		Files.write(PageRankData.EDGES, edgesFile, Charsets.UTF_8);
+
+		verticesPath = verticesFile.toURI().toString();
+		edgesPath = edgesFile.toURI().toString();
+	}
+
+	@After
+	public void after() throws Exception{
+		compareKeyValuePairsWithDelta(expected, resultPath, " ", 0.01);
+	}
+
+	@Test
+	public void testPageRankSmallNumberOfIterations() throws Exception {
+		PageRank.main(new String[]{
+				"--pages", verticesPath,
+				"--links", edgesPath,
+				"--output", resultPath,
+				"--numPages", PageRankData.NUM_VERTICES + "",
+				"--iterations", "3"});
+		expected =  PageRankData.RANKS_AFTER_3_ITERATIONS;
+	}
+
+	@Test
+	public void testPageRankWithConvergenceCriterion() throws Exception {
+		PageRank.main(new String[]{
+				"--pages", verticesPath,
+				"--links", edgesPath,
+				"--output", resultPath,
+				"--numPages", PageRankData.NUM_VERTICES + "",
+				"--vertices", "1000"});
+		expected = PageRankData.RANKS_AFTER_EPSILON_0_0001_CONVERGENCE;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java
new file mode 100644
index 0000000..fc5c6a7
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/TransitiveClosureITCase.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.examples.java.graph.TransitiveClosureNaive;
+import org.apache.flink.test.testdata.ConnectedComponentsData;
+import org.apache.flink.test.testdata.TransitiveClosureData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.BufferedReader;
+
+/**
+ * Test for {@link TransitiveClosureNaive}.
+ */
+public class TransitiveClosureITCase extends JavaProgramTestBase {
+
+	private static final long SEED = 0xBADC0FFEEBEEFL;
+
+	private static final int NUM_VERTICES = 100;
+
+	private static final int NUM_EDGES = 500;
+
+	private String edgesPath;
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
+		resultPath = getTempFilePath("results");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TransitiveClosureNaive.main(
+				"--edges", edgesPath,
+				"--output", resultPath,
+				"--iterations", "5");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		for (BufferedReader reader : getResultReader(resultPath)) {
+			TransitiveClosureData.checkOddEvenResult(reader);
+		}
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java
new file mode 100644
index 0000000..555edc1
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/java/WebLogAnalysisITCase.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.test.example.java;
+
+import org.apache.flink.examples.java.relational.WebLogAnalysis;
+import org.apache.flink.test.testdata.WebLogAnalysisData;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+/**
+ * Test for {@link WebLogAnalysis}.
+ */
+public class WebLogAnalysisITCase extends JavaProgramTestBase {
+
+	private String docsPath;
+	private String ranksPath;
+	private String visitsPath;
+	private String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		docsPath = createTempFile("docs", WebLogAnalysisData.DOCS);
+		ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS);
+		visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WebLogAnalysis.main(new String[] {
+				"--documents", docsPath,
+				"--ranks", ranksPath,
+				"--visits", visitsPath,
+				"--output", resultPath});
+	}
+}


[14/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
deleted file mode 100644
index 6f93722..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
+++ /dev/null
@@ -1,1636 +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.javaApiOperators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.RichGroupReduceFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CrazyNested;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.FromTupleWithCTor;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.PojoContainingTupleAndWritable;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.joda.time.DateTime;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import scala.math.BigInt;
-
-import java.util.Collection;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-
-@SuppressWarnings({"serial", "unchecked", "UnusedDeclaration"})
-@RunWith(Parameterized.class)
-public class GroupReduceITCase extends MultipleProgramsTestBase {
-
-	public GroupReduceITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testCorrectnessofGroupReduceOnTupleContainingPrimitiveByteArrayWithKeyFieldSelectors() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<byte[], Integer>> ds = CollectionDataSets.getTuple2WithByteArrayDataSet(env);
-		DataSet<Integer> reduceDs = ds.
-				groupBy(0).reduceGroup(new ByteArrayGroupReduce());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "0\n"
-				+ "1\n"
-				+ "2\n"
-				+ "3\n"
-				+ "4\n";
-
-		compareResultAsText(result, expected);
-
-	}
-
-	public static class ByteArrayGroupReduce implements GroupReduceFunction<Tuple2<byte[], Integer>, Integer> {
-		@Override
-		public void reduce(Iterable<Tuple2<byte[], Integer>> values, Collector<Integer> out) throws Exception {
-			int sum = 0;
-			for (Tuple2<byte[], Integer> value : values) {
-				sum += value.f1;
-			}
-			out.collect(sum);
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector() throws Exception{
-		/*
-		 * check correctness of groupReduce on tuples with key field selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> reduceDs = ds.
-				groupBy(1).reduceGroup(new Tuple3GroupReduce());
-
-		List<Tuple2<Integer, Long>> result = reduceDs.collect();
-
-		String expected = "1,1\n" +
-				"5,2\n" +
-				"15,3\n" +
-				"34,4\n" +
-				"65,5\n" +
-				"111,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() throws Exception {
-		/*
-		 * check correctness of groupReduce on tuples with multiple key field selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
-				groupBy(4, 0).reduceGroup(new Tuple5GroupReduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
-
-		String expected = "1,1,0,P-),1\n" +
-				"2,3,0,P-),1\n" +
-				"2,2,0,P-),2\n" +
-				"3,9,0,P-),2\n" +
-				"3,6,0,P-),3\n" +
-				"4,17,0,P-),1\n" +
-				"4,17,0,P-),2\n" +
-				"5,11,0,P-),1\n" +
-				"5,29,0,P-),2\n" +
-				"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting() throws Exception {
-		/*
-		 * check correctness of groupReduce on tuples with key field selector and group sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup(new Tuple3SortedGroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n"
-				+
-				"5,2,Hello-Hello world\n" +
-				"15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" +
-				"34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" +
-				"65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" +
-				"111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithKeyExtractor() throws Exception {
-		/*
-		 * check correctness of groupReduce on tuples with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> reduceDs = ds.
-				groupBy(new KeySelector1()).reduceGroup(new Tuple3GroupReduce());
-
-		List<Tuple2<Integer, Long>> result = reduceDs.collect();
-
-		String expected = "1,1\n" +
-				"5,2\n" +
-				"15,3\n" +
-				"34,4\n" +
-				"65,5\n" +
-				"111,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector1 implements KeySelector<Tuple3<Integer, Long, String>, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long getKey(Tuple3<Integer, Long, String> in) {
-			return in.f1;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnCustomTypeWithTypeExtractor() throws Exception {
-		/*
-		 * check correctness of groupReduce on custom type with type extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds.
-				groupBy(new KeySelector2()).reduceGroup(new CustomTypeGroupReduce());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "1,0,Hello!\n" +
-				"2,3,Hello!\n" +
-				"3,12,Hello!\n" +
-				"4,30,Hello!\n" +
-				"5,60,Hello!\n" +
-				"6,105,Hello!\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfAllGroupReduceForTuples() throws Exception {
-		/*
-		 * check correctness of all-groupreduce for tuples
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.reduceGroup(new AllAddingTuple3GroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "231,91,Hello World\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfAllGroupReduceForCustomTypes() throws Exception {
-		/*
-		 * check correctness of all-groupreduce for custom types
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "91,210,Hello!";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceWithBroadcastSet() throws Exception {
-		/*
-		 * check correctness of groupReduce with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).reduceGroup(new BCTuple3GroupReduce()).withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,55\n" +
-				"5,2,55\n" +
-				"15,3,55\n" +
-				"34,4,55\n" +
-				"65,5,55\n" +
-				"111,6,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceIfUDFReturnsInputObjectsMultipleTimesWhileChangingThem() throws Exception{
-		/*
-		 * check correctness of groupReduce if UDF returns input objects multiple times and changes it in between
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "11,1,Hi!\n" +
-				"21,1,Hi again!\n" +
-				"12,2,Hi!\n" +
-				"22,2,Hi again!\n" +
-				"13,2,Hi!\n" +
-				"23,2,Hi again!\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() throws Exception {
-		/*
-		 * check correctness of groupReduce on custom type with key extractor and combine
-		 */
-		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds.
-				groupBy(new KeySelector3()).reduceGroup(new CustomTypeGroupReduceWithCombine());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "1,0,test1\n" +
-				"2,3,test2\n" +
-				"3,12,test3\n" +
-				"4,30,test4\n" +
-				"5,60,test5\n" +
-				"6,105,test6\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector3 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception {
-		/*
-		 * check correctness of groupReduce on tuples with combine
-		 */
-		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(2); // important because it determines how often the combiner is called
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, String>> reduceDs = ds.
-				groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine());
-
-		List<Tuple2<Integer, String>> result = reduceDs.collect();
-
-		String expected = "1,test1\n" +
-				"5,test2\n" +
-				"15,test3\n" +
-				"34,test4\n" +
-				"65,test5\n" +
-				"111,test6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfAllGroupReduceForTuplesWithCombine() throws Exception {
-		/*
-		 * check correctness of all-groupreduce for tuples with combine
-		 */
-		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env)
-				.map(new IdentityMapper<Tuple3<Integer, Long, String>>()).setParallelism(4);
-
-		Configuration cfg = new Configuration();
-		cfg.setString(Optimizer.HINT_SHIP_STRATEGY, Optimizer.HINT_SHIP_STRATEGY_REPARTITION);
-		DataSet<Tuple2<Integer, String>> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine())
-				.withParameters(cfg);
-
-		List<Tuple2<Integer, String>> result = reduceDs.collect();
-
-		String expected = "322," +
-				"testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupreduceWithDescendingGroupSort() throws Exception {
-		/*
-		 * check correctness of groupReduce with descending group sort
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n"
-				+
-				"5,2,Hello world-Hello\n" +
-				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
-				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
-				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
-				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() throws Exception {
-		/*
-		 * check correctness of groupReduce on tuples with tuple-returning key selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
-				groupBy(new KeySelector4()).reduceGroup(new Tuple5GroupReduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
-
-		String expected = "1,1,0,P-),1\n" +
-				"2,3,0,P-),1\n" +
-				"2,2,0,P-),2\n" +
-				"3,9,0,P-),2\n" +
-				"3,6,0,P-),3\n" +
-				"4,17,0,P-),1\n" +
-				"4,17,0,P-),2\n" +
-				"5,11,0,P-),1\n" +
-				"5,29,0,P-),2\n" +
-				"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector4 implements KeySelector<Tuple5<Integer,Long,Integer,String,Long>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple5<Integer,Long,Integer,String,Long> t) {
-			return new Tuple2<>(t.f0, t.f4);
-		}
-	}
-
-	@Test
-	public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting() throws Exception {
-		/*
-		 * check that input of combiner is also sorted for combinable groupReduce with group sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).sortGroup(0, Order.ASCENDING).reduceGroup(new OrderCheckingCombinableReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"4,3,Hello world, how are you?\n" +
-				"7,4,Comment#1\n" +
-				"11,5,Comment#5\n" +
-				"16,6,Comment#10\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testDeepNesting() throws Exception {
-		/*
-		 * Deep nesting test
-		 * + null value in pojo
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CrazyNested> ds = CollectionDataSets.getCrazyNestedDataSet(env);
-		DataSet<Tuple2<String, Integer>> reduceDs = ds.groupBy("nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal")
-				.reduceGroup(new GroupReducer1());
-
-		List<Tuple2<String, Integer>> result = reduceDs.collect();
-
-		String expected = "aa,1\nbb,2\ncc,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class GroupReducer1 implements GroupReduceFunction<CollectionDataSets.CrazyNested, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<CrazyNested> values,
-				Collector<Tuple2<String, Integer>> out)
-						throws Exception {
-			int c = 0; String n = null;
-			for(CrazyNested v : values) {
-				c++; // haha
-				n = v.nest_Lvl1.nest_Lvl2.nest_Lvl3.nest_Lvl4.f1nal;
-			}
-			out.collect(new Tuple2<>(n,c));
-		}
-	}
-
-	@Test
-	public void testPojoExtendingFromTupleWithCustomFields() throws Exception {
-		/*
-		 * Test Pojo extending from tuple WITH custom fields
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<FromTupleWithCTor> ds = CollectionDataSets.getPojoExtendingFromTuple(env);
-		DataSet<Integer> reduceDs = ds.groupBy("special", "f2")
-				.reduceGroup(new GroupReducer2());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "3\n2\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class GroupReducer2 implements GroupReduceFunction<FromTupleWithCTor, Integer> {
-
-		@Override
-		public void reduce(Iterable<FromTupleWithCTor> values, Collector<Integer> out) {
-			out.collect(countElements(values));
-		}
-	}
-
-	@Test
-	public void testPojoContainigWritableAndTuples() throws Exception {
-		/*
-		 * Test Pojo containing a Writable and Tuples
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<PojoContainingTupleAndWritable> ds = CollectionDataSets.getPojoContainingTupleAndWritable(env);
-		DataSet<Integer> reduceDs = ds.groupBy("hadoopFan", "theTuple.*") // full tuple selection
-				.reduceGroup(new GroupReducer3());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "1\n5\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class GroupReducer3 implements GroupReduceFunction<PojoContainingTupleAndWritable, Integer> {
-
-		@Override
-		public void reduce(Iterable<PojoContainingTupleAndWritable> values, Collector<Integer> out) {
-			out.collect(countElements(values));
-		}
-	}
-
-	@Test
-	public void testTupleContainingPojosAndRegularFields() throws Exception {
-		/*
-		 * Test Tuple containing pojos and regular fields
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer,CrazyNested, POJO>> ds = CollectionDataSets.getTupleContainingPojos(env);
-
-		DataSet<Integer> reduceDs = ds.groupBy("f0", "f1.*") // nested full tuple selection
-				.reduceGroup(new GroupReducer4());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "3\n1\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class GroupReducer4 implements GroupReduceFunction<Tuple3<Integer,CrazyNested, POJO>, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public void reduce(Iterable<Tuple3<Integer,CrazyNested, POJO>> values, Collector<Integer> out) {
-			out.collect(countElements(values));
-		}
-	}
-
-	@Test
-	public void testStringBasedDefinitionOnGroupSort() throws Exception {
-		/*
-		 * Test string-based definition on group sort, based on test:
-		 * check correctness of groupReduce with descending group sort
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).sortGroup("f2", Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n"
-				+
-				"5,2,Hello world-Hello\n" +
-				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
-				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
-				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
-				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testIntBasedDefinitionOnGroupSortForFullNestedTuple() throws Exception {
-		/*
-		 * Test int-based definition on group sort, for (full) nested Tuple
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
-		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup(0, Order.DESCENDING).reduceGroup(new NestedTupleReducer());
-		List<String> result = reduceDs.collect();
-
-		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
-				"b--(2,2)-\n"+
-				"c--(4,9)-(3,6)-(3,3)-\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testIntBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception {
-		/*
-		 * Test int-based definition on group sort, for (partial) nested Tuple ASC
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("f1")
-				.sortGroup("f0.f0", Order.ASCENDING)
-				.sortGroup("f0.f1", Order.ASCENDING)
-				.reduceGroup(new NestedTupleReducer());
-		List<String> result = reduceDs.collect();
-
-		String expected = "a--(1,2)-(1,3)-(2,1)-\n" +
-				"b--(2,2)-\n"+
-				"c--(3,3)-(3,6)-(4,9)-\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testStringBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception {
-		/*
-		 * Test string-based definition on group sort, for (partial) nested Tuple DESC
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).reduceGroup(new NestedTupleReducer());
-		List<String> result = reduceDs.collect();
-
-		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
-				"b--(2,2)-\n"+
-				"c--(4,9)-(3,3)-(3,6)-\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeys() throws Exception {
-		/*
-		 * Test string-based definition on group sort, for two grouping keys
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).sortGroup("f0.f1", Order.DESCENDING).reduceGroup(new NestedTupleReducer());
-		List<String> result = reduceDs.collect();
-
-		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
-				"b--(2,2)-\n"+
-				"c--(4,9)-(3,6)-(3,3)-\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos() throws Exception {
-		/*
-		 * Test string-based definition on group sort, for two grouping keys with Pojos
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<PojoContainingTupleAndWritable> ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env);
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("hadoopFan").sortGroup("theTuple.f0", Order.DESCENDING).sortGroup("theTuple.f1", Order.DESCENDING)
-				.reduceGroup(new GroupReducer5());
-		List<String> result = reduceDs.collect();
-
-		String expected = "1---(10,100)-\n"
-				+
-				"2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testTupleKeySelectorGroupSort() throws Exception {
-		/*
-		 * check correctness of sorted groupReduce on tuples with keyselector sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
-				.groupBy(new LongFieldExtractor<Tuple3<Integer, Long, String>>(1))
-				.sortGroup(new StringFieldExtractor<Tuple3<Integer, Long, String>>(2), Order.DESCENDING)
-				.reduceGroup(new Tuple3SortedGroupReduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n"
-				+
-				"5,2,Hello world-Hello\n" +
-				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
-				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
-				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
-				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class TwoTuplePojoExtractor implements KeySelector<CustomType, Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> getKey(CustomType value) throws Exception {
-			return new Tuple2<>(value.myInt, value.myInt);
-		}
-	}
-
-	public static class StringPojoExtractor implements KeySelector<CustomType, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(CustomType value) throws Exception {
-			return value.myString;
-		}
-	}
-
-	@Test
-	public void testPojoKeySelectorGroupSort() throws Exception {
-		/*
-		 * check correctness of sorted groupReduce on custom type with keyselector sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds
-				.groupBy(new TwoTuplePojoExtractor())
-				.sortGroup(new StringPojoExtractor(), Order.DESCENDING)
-				.reduceGroup(new CustomTypeSortedGroupReduce());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "1,0,Hi\n"
-				+
-				"2,3,Hello world-Hello\n" +
-				"3,12,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
-				"4,30,Comment#4-Comment#3-Comment#2-Comment#1\n" +
-				"5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
-				"6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class LongFieldExtractor<T extends Tuple>  implements KeySelector<T, Long> {
-		private static final long serialVersionUID = 1L;
-		private int field;
-
-		
-		public LongFieldExtractor() { }
-
-		public LongFieldExtractor(int field) {
-			this.field = field;
-		}
-
-		@Override
-		public Long getKey(T t) throws Exception {
-			return ((Tuple)t).getField(field);
-		}
-	}
-
-	public static class IntFieldExtractor<T extends Tuple>  implements KeySelector<T, Integer> {
-		private static final long serialVersionUID = 1L;
-		private int field;
-
-		public IntFieldExtractor() { }
-
-		public IntFieldExtractor(int field) {
-			this.field = field;
-		}
-
-		@Override
-		public Integer getKey(T t) throws Exception {
-			return ((Tuple)t).getField(field);
-		}
-	}
-
-	public static class StringFieldExtractor<T extends Tuple>  implements KeySelector<T, String> {
-		private static final long serialVersionUID = 1L;
-		private int field;
-
-		public StringFieldExtractor() { }
-
-		public StringFieldExtractor(int field) {
-			this.field = field;
-		}
-
-		@Override
-		public String getKey(T t) throws Exception {
-			return t.getField(field);
-		}
-	}
-
-	@Test
-	public void testTupleKeySelectorSortWithCombine() throws Exception {
-		/*
-		 * check correctness of sorted groupReduce with combine on tuples with keyselector sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, String>> reduceDs = ds.
-				groupBy(new LongFieldExtractor<Tuple3<Integer, Long, String>>(1))
-				.sortGroup(new StringFieldExtractor<Tuple3<Integer, Long, String>>(2), Order.DESCENDING)
-				.reduceGroup(new Tuple3SortedGroupReduceWithCombine());
-
-		List<Tuple2<Integer, String>> result = reduceDs.collect();
-
-		if (super.mode != TestExecutionMode.COLLECTION) {
-			String expected = "1,Hi\n"
-					+
-					"5,Hello world-Hello\n" +
-					"15,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
-					"34,Comment#4-Comment#3-Comment#2-Comment#1\n" +
-					"65,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
-					"111,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
-
-			compareResultAsTuples(result, expected);
-		}
-	}
-
-	public static class FiveToTwoTupleExtractor implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Long, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, Integer, String, Long> in) {
-			return new Tuple2<>(in.f4, in.f2);
-		}
-	}
-
-	@Test
-	public void testTupleKeySelectorSortCombineOnTuple() throws Exception {
-		/*
-		 * check correctness of sorted groupReduceon with Tuple2 keyselector sorting
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds
-				.groupBy(new IntFieldExtractor<Tuple5<Integer, Long, Integer, String, Long>>(0))
-				.sortGroup(new FiveToTwoTupleExtractor(), Order.DESCENDING)
-				.reduceGroup(new Tuple5SortedGroupReduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
-
-		String expected = "1,1,0,Hallo,1\n"
-				+
-				"2,5,0,Hallo Welt-Hallo Welt wie,1\n" +
-				"3,15,0,BCD-ABC-Hallo Welt wie gehts?,2\n" +
-				"4,34,0,FGH-CDE-EFG-DEF,1\n" +
-				"5,65,0,IJK-HIJ-KLM-JKL-GHI,1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class GroupReducer5 implements GroupReduceFunction<CollectionDataSets.PojoContainingTupleAndWritable, String> {
-		@Override
-		public void reduce(
-				Iterable<PojoContainingTupleAndWritable> values,
-				Collector<String> out) throws Exception {
-			boolean once = false;
-			StringBuilder concat = new StringBuilder();
-			for(PojoContainingTupleAndWritable value : values) {
-				if(!once) {
-					concat.append(value.hadoopFan.get());
-					concat.append("---");
-					once = true;
-				}
-				concat.append(value.theTuple);
-				concat.append("-");
-			}
-			out.collect(concat.toString());
-		}
-	}
-
-	@Test
-	public void testGroupingWithPojoContainingMultiplePojos() throws Exception {
-		/*
-		 * Test grouping with pojo containing multiple pojos (was a bug)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<CollectionDataSets.PojoWithMultiplePojos> ds = CollectionDataSets.getPojoWithMultiplePojos(env);
-
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("p2.a2")
-				.reduceGroup(new GroupReducer6());
-		List<String> result = reduceDs.collect();
-
-		String expected = "b\nccc\nee\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class GroupReducer6 implements GroupReduceFunction<CollectionDataSets.PojoWithMultiplePojos, String> {
-		@Override
-		public void reduce(
-				Iterable<CollectionDataSets.PojoWithMultiplePojos> values,
-				Collector<String> out) throws Exception {
-			StringBuilder concat = new StringBuilder();
-			for(CollectionDataSets.PojoWithMultiplePojos value : values) {
-				concat.append(value.p2.a2);
-			}
-			out.collect(concat.toString());
-		}
-	}
-
-	@Test
-	public void testJavaCollectionsWithinPojos() throws Exception {
-		/*
-		 * Test Java collections within pojos ( == test kryo)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("key")
-				.reduceGroup(new GroupReducer7());
-		List<String> result = reduceDs.collect();
-
-		String expected = "callFor key 0 we got: pojo.a=apojo.a=bFor key 0 we got: pojo.a=a2pojo.a=b2\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class GroupReducer7 implements GroupReduceFunction<CollectionDataSets.PojoWithCollection, String> {
-		
-		@Override
-		public void reduce(Iterable<CollectionDataSets.PojoWithCollection> values, Collector<String> out) {
-			StringBuilder concat = new StringBuilder();
-			concat.append("call");
-			for (CollectionDataSets.PojoWithCollection value : values) {
-				concat.append("For key ").append(value.key).append(" we got: ");
-				
-				for (CollectionDataSets.Pojo1 p :value.pojos) {
-					concat.append("pojo.a=").append(p.a);
-				}
-			}
-			out.collect(concat.toString());
-		}
-	}
-
-	@Test
-	public void testGroupByGenericType() throws Exception {
-		/*
-		 * Group by generic type
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
-
-		// f0.f0 is first integer
-		DataSet<String> reduceDs = ds.groupBy("bigInt")
-				.reduceGroup(new GroupReducer8());
-		List<String> result = reduceDs.collect();
-		ExecutionConfig ec = env.getConfig();
-
-		// check if automatic type registration with Kryo worked
-		Assert.assertTrue(ec.getRegisteredKryoTypes().contains(BigInt.class));
-		Assert.assertFalse(ec.getRegisteredKryoTypes().contains(java.sql.Date.class));
-
-		String expected = null;
-
-		String localExpected = "[call\n" +
-				"For key 92233720368547758070 we got:\n" +
-				"PojoWithCollection{pojos.size()=2, key=0, sqlDate=2033-05-18, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=10, mixed=[{someKey=1}, /this/is/wrong, uhlala]}\n" +
-				"For key 92233720368547758070 we got:\n" +
-				"PojoWithCollection{pojos.size()=2, key=0, sqlDate=1976-05-03, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=31104000, mixed=null}]";
-
-		Assert.assertEquals(localExpected, result.toString());
-	}
-
-	@Test
-	public void testGroupReduceSelectorKeysWithSemProps() throws Exception {
-
-		/*
-		 * Test that semantic properties are correctly adapted when using Selector Keys
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> reduceDs = ds
-				// group by selector key
-				.groupBy(new KeySelector<Tuple5<Integer,Long,Integer,String,Long>, Long>() {
-					@Override
-					public Long getKey(Tuple5<Integer, Long, Integer, String, Long> v) throws Exception {
-						return (v.f0*v.f1)-(v.f2*v.f4);
-					}
-				})
-				.reduceGroup(
-						new GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>>() {
-							@Override
-							public void reduce(Iterable<Tuple5<Integer, Long, Integer, String, Long>> values, Collector<Tuple5<Integer, Long, Integer, String, Long>> out) throws Exception {
-								for (Tuple5<Integer, Long, Integer, String, Long> v : values) {
-									out.collect(v);
-								}
-							}
-						})
-						// add forward field information
-						.withForwardedFields("0")
-						// group again and reduce
-						.groupBy(0).reduceGroup(
-								new GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>>() {
-									@Override
-									public void reduce(Iterable<Tuple5<Integer, Long, Integer, String, Long>> values, Collector<Tuple2<Integer, Long>> out) throws Exception {
-										int k = 0;
-										long s = 0;
-										for (Tuple5<Integer, Long, Integer, String, Long> v : values) {
-											k = v.f0;
-											s += v.f1;
-										}
-										out.collect(new Tuple2<>(k, s));
-									}
-								}
-								);
-
-		List<Tuple2<Integer, Long>> result = reduceDs.collect();
-
-		String expected = "1,1\n" +
-				"2,5\n" +
-				"3,15\n" +
-				"4,34\n" +
-				"5,65\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testGroupReduceWithAtomicValue() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Integer> ds = env.fromElements(1, 1, 2, 3, 4);
-		DataSet<Integer> reduceDs = ds.groupBy("*").reduceGroup(new GroupReduceFunction<Integer, Integer>() {
-			@Override
-			public void reduce(Iterable<Integer> values, Collector<Integer> out) throws Exception {
-				out.collect(values.iterator().next());
-			}
-		});
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "1\n" +
-				"2\n" +
-				"3\n" +
-				"4";
-
-		compareResultAsText(result, expected);
-	}
-
-	/**
-	 * Fix for FLINK-2019.
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testJodatimeDateTimeWithKryo() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Tuple2<Integer, DateTime>> ds = env.fromElements(new Tuple2<>(1, DateTime.now()));
-		DataSet<Tuple2<Integer, DateTime>> reduceDs = ds.groupBy("f1").sum(0).project(0);
-
-		List<Tuple2<Integer, DateTime>> result = reduceDs.collect();
-
-		String expected = "1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	/**
-	 * Fix for FLINK-2158.
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testDateNullException() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Integer, Date>> in = env.fromElements(new Tuple2<>(0, new Date(1230000000)),
-				new Tuple2<Integer, Date>(1, null),
-				new Tuple2<>(2, new Date(1230000000))
-		);
-
-		DataSet<String> r = in.groupBy(0).reduceGroup(new GroupReduceFunction<Tuple2<Integer, Date>, String>() {
-			@Override
-			public void reduce(Iterable<Tuple2<Integer, Date>> values, Collector<String> out) throws Exception {
-				for (Tuple2<Integer, Date> e : values) {
-					out.collect(Integer.toString(e.f0));
-				}
-			}
-		});
-
-		List<String> result = r.collect();
-
-		String expected = "0\n1\n2\n";
-		compareResultAsText(result, expected);
-	}
-
-
-
-	public static class GroupReducer8 implements GroupReduceFunction<CollectionDataSets.PojoWithCollection, String> {
-		@Override
-		public void reduce(Iterable<CollectionDataSets.PojoWithCollection> values, Collector<String> out) {
-			StringBuilder concat = new StringBuilder();
-			concat.append("call");
-			for (CollectionDataSets.PojoWithCollection value : values) {
-				concat.append("\nFor key ").append(value.bigInt).append(" we got:\n").append(value);
-			}
-			out.collect(concat.toString());
-		}
-	}
-
-	public static class NestedTupleReducer implements GroupReduceFunction<Tuple2<Tuple2<Integer,Integer>,String>, String> {
-		@Override
-		public void reduce(Iterable<Tuple2<Tuple2<Integer, Integer>, String>> values, Collector<String> out) {
-			boolean once = false;
-			StringBuilder concat = new StringBuilder();
-			for(Tuple2<Tuple2<Integer, Integer>, String> value : values) {
-				if(!once) {
-					concat.append(value.f1).append("--");
-					once = true;
-				}
-				concat.append(value.f0); // the tuple with the sorted groups
-				concat.append("-");
-			}
-			out.collect(concat.toString());
-		}
-	}
-
-	public static class Tuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, Long>> out) {
-			int i = 0;
-			long l = 0L;
-
-			for (Tuple3<Integer, Long, String> t : values) {
-				i += t.f0;
-				l = t.f1;
-			}
-
-			out.collect(new Tuple2<>(i, l));
-
-		}
-	}
-
-	public static class Tuple3SortedGroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-			int sum = 0;
-			long key = 0;
-			StringBuilder concat = new StringBuilder();
-
-			for (Tuple3<Integer, Long, String> next : values) {
-				sum += next.f0;
-				key = next.f1;
-				concat.append(next.f2).append("-");
-			}
-
-			if (concat.length() > 0) {
-				concat.setLength(concat.length() - 1);
-			}
-
-			out.collect(new Tuple3<>(sum, key, concat.toString()));
-		}
-	}
-
-	public static class Tuple5GroupReduce implements GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> values,
-				Collector<Tuple5<Integer, Long, Integer, String, Long>> out)
-		{
-			int i = 0;
-			long l = 0L;
-			long l2 = 0L;
-
-			for ( Tuple5<Integer, Long, Integer, String, Long> t : values ) {
-				i = t.f0;
-				l += t.f1;
-				l2 = t.f4;
-			}
-
-			out.collect(new Tuple5<>(i, l, 0, "P-)", l2));
-		}
-	}
-
-	public static class Tuple5SortedGroupReduce implements GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> values,
-				Collector<Tuple5<Integer, Long, Integer, String, Long>> out)
-		{
-			int i = 0;
-			long l = 0L;
-			long l2 = 0L;
-			StringBuilder concat = new StringBuilder();
-
-			for ( Tuple5<Integer, Long, Integer, String, Long> t : values ) {
-				i = t.f0;
-				l += t.f1;
-				concat.append(t.f3).append("-");
-				l2 = t.f4;
-			}
-			if (concat.length() > 0) {
-				concat.setLength(concat.length() - 1);
-			}
-
-			out.collect(new Tuple5<>(i, l, 0, concat.toString(), l2));
-		}
-	}
-
-	public static class CustomTypeGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-
-		@Override
-		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
-			final Iterator<CustomType> iter = values.iterator();
-
-			CustomType o = new CustomType();
-			CustomType c = iter.next();
-
-			o.myString = "Hello!";
-			o.myInt = c.myInt;
-			o.myLong = c.myLong;
-
-			while (iter.hasNext()) {
-				CustomType next = iter.next();
-				o.myLong += next.myLong;
-			}
-
-			out.collect(o);
-
-		}
-	}
-
-	public static class CustomTypeSortedGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-
-		@Override
-		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
-			final Iterator<CustomType> iter = values.iterator();
-
-			CustomType o = new CustomType();
-			CustomType c = iter.next();
-
-			StringBuilder concat = new StringBuilder(c.myString);
-			o.myInt = c.myInt;
-			o.myLong = c.myLong;
-
-			while (iter.hasNext()) {
-				CustomType next = iter.next();
-				concat.append("-").append(next.myString);
-				o.myLong += next.myLong;
-
-			}
-
-			o.myString = concat.toString();
-			out.collect(o);
-
-		}
-	}
-
-	public static class InputReturningTuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-
-				if(t.f0 < 4) {
-					t.f2 = "Hi!";
-					t.f0 += 10;
-					out.collect(t);
-					t.f0 += 10;
-					t.f2 = "Hi again!";
-					out.collect(t);
-				}
-			}
-		}
-	}
-
-	public static class AllAddingTuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			int i = 0;
-			long l = 0L;
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				i += t.f0;
-				l += t.f1;
-			}
-
-			out.collect(new Tuple3<>(i, l, "Hello World"));
-		}
-	}
-
-	public static class AllAddingCustomTypeGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
-
-			CustomType o = new CustomType(0, 0, "Hello!");
-
-			for (CustomType next : values) {
-				o.myInt += next.myInt;
-				o.myLong += next.myLong;
-			}
-
-			out.collect(o);
-		}
-	}
-
-	public static class BCTuple3GroupReduce extends RichGroupReduceFunction<Tuple3<Integer, Long, String>,Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private String f2Replace = "";
-
-		@Override
-		public void open(Configuration config) {
-
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			f2Replace = sum+"";
-
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			int i = 0;
-			long l = 0L;
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				i += t.f0;
-				l = t.f1;
-			}
-
-			out.collect(new Tuple3<>(i, l, this.f2Replace));
-
-		}
-	}
-
-	public static class Tuple3GroupReduceWithCombine
-		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
-					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>
-	{
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			Tuple3<Integer, Long, String> o = new Tuple3<>(0, 0L, "");
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				o.f0 += t.f0;
-				o.f1 = t.f1;
-				o.f2 = "test"+o.f1;
-			}
-
-			out.collect(o);
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
-
-			int i = 0;
-			String s = "";
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				i += t.f0;
-				s = t.f2;
-			}
-
-			out.collect(new Tuple2<>(i, s));
-
-		}
-	}
-
-	public static class Tuple3SortedGroupReduceWithCombine
-		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
-					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-			int sum = 0;
-			long key = 0;
-			StringBuilder concat = new StringBuilder();
-
-			for (Tuple3<Integer, Long, String> next : values) {
-				sum += next.f0;
-				key = next.f1;
-				concat.append(next.f2).append("-");
-			}
-
-			if (concat.length() > 0) {
-				concat.setLength(concat.length() - 1);
-			}
-
-			out.collect(new Tuple3<>(sum, key, concat.toString()));
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
-			int i = 0;
-			String s = "";
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				i += t.f0;
-				s = t.f2;
-			}
-
-			out.collect(new Tuple2<>(i, s));
-		}
-	}
-
-	public static class Tuple3AllGroupReduceWithCombine
-		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
-					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			Tuple3<Integer, Long, String> o = new Tuple3<>(0, 0L, "");
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				o.f0 += t.f0;
-				o.f1 += t.f1;
-				o.f2 += "test";
-			}
-
-			out.collect(o);
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
-
-			int i = 0;
-			String s = "";
-
-			for ( Tuple3<Integer, Long, String> t : values ) {
-				i += t.f0 + t.f1;
-				s += t.f2;
-			}
-
-			out.collect(new Tuple2<>(i, s));
-
-		}
-	}
-
-	public static class CustomTypeGroupReduceWithCombine
-		implements GroupReduceFunction<CustomType, CustomType>,
-					GroupCombineFunction<CustomType, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void combine(Iterable<CustomType> values, Collector<CustomType> out) throws Exception {
-
-			CustomType o = new CustomType();
-
-			for ( CustomType c : values ) {
-				o.myInt = c.myInt;
-				o.myLong += c.myLong;
-				o.myString = "test"+c.myInt;
-			}
-
-			out.collect(o);
-		}
-
-		@Override
-		public void reduce(Iterable<CustomType> values, Collector<CustomType> out)  {
-
-			CustomType o = new CustomType(0, 0, "");
-
-			for ( CustomType c : values) {
-				o.myInt = c.myInt;
-				o.myLong += c.myLong;
-				o.myString = c.myString;
-			}
-
-			out.collect(o);
-
-		}
-	}
-
-	public static class OrderCheckingCombinableReduce
-		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>,
-					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			Iterator<Tuple3<Integer,Long,String>> it = values.iterator();
-			Tuple3<Integer,Long,String> t = it.next();
-
-			int i = t.f0;
-			out.collect(t);
-
-			while(it.hasNext()) {
-				t = it.next();
-				if(i > t.f0 || t.f2.equals("INVALID-ORDER!")) {
-					t.f2 = "INVALID-ORDER!";
-					out.collect(t);
-				}
-			}
-		}
-
-		@Override
-		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
-
-			Iterator<Tuple3<Integer,Long,String>> it = values.iterator();
-			Tuple3<Integer,Long,String> t = it.next();
-
-			int i = t.f0;
-			out.collect(t);
-
-			while(it.hasNext()) {
-				t = it.next();
-				if(i > t.f0) {
-					t.f2 = "INVALID-ORDER!";
-					out.collect(t);
-				}
-			}
-
-		}
-
-
-	}
-
-	public static final class IdentityMapper<T> extends RichMapFunction<T, T> {
-		@Override
-		public T map(T value) { return value; }
-	}
-
-	private static int countElements(Iterable<?> iterable) {
-		int c = 0;
-		for (@SuppressWarnings("unused") Object o : iterable) {
-			c++;
-		}
-		return c;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
deleted file mode 100644
index f47d3e3..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/JoinITCase.java
+++ /dev/null
@@ -1,938 +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.javaApiOperators;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.distributions.DataDistribution;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.common.functions.RichFlatJoinFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.utils.DataSetUtils;
-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.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class JoinITCase extends MultipleProgramsTestBase {
-
-	public JoinITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Test
-	public void testUDFJoinOnTuplesWithKeyFieldPositions() throws Exception {
-		/*
-		 * UDF Join on tuples with key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(1)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testeUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception {
-		/*
-		 * UDF Join on tuples with multiple key field positions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.join(ds2)
-						.where(0, 1)
-						.equalTo(0, 4)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt wie gehts?\n" +
-				"Hello world,ABC\n" +
-				"I am fine.,HIJ\n" +
-				"I am fine.,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testDefaultJoinOnTuples() throws Exception {
-		/*
-		 * Default Join on tuples
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> joinDs =
-				ds1.join(ds2)
-						.where(0)
-						.equalTo(2);
-
-		List<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> result = joinDs.collect();
-
-		String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" +
-				"(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" +
-				"(3,2,Hello world),(3,4,3,Hallo Welt wie gehts?,2)\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	@Test
-	public void testJoinWithHuge() throws Exception {
-		/*
-		 * Join with Huge
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs = ds1.joinWithHuge(ds2)
-				.where(1)
-				.equalTo(1)
-				.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithTiny() throws Exception {
-		/*
-		 * Join with Tiny
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.joinWithTiny(ds2)
-						.where(1)
-						.equalTo(1)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinThatReturnsTheLeftInputObject() throws Exception {
-		/*
-		 * Join that returns the left input object
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(1)
-						.with(new LeftReturningJoin());
-
-		List<Tuple3<Integer, Long, String>> result = joinDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinThatReturnsTheRightInputObject() throws Exception {
-		/*
-		 * Join that returns the right input object
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(1)
-						.with(new RightReturningJoin());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = joinDs.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"2,2,1,Hallo Welt,2\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithBroadcastSet() throws Exception {
-		/*
-		 * Join with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple3<String, String, Integer>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(4)
-						.with(new T3T5BCJoin())
-						.withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<String, String, Integer>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo,55\n" +
-				"Hi,Hallo Welt wie,55\n" +
-				"Hello,Hallo Welt,55\n" +
-				"Hello world,Hallo Welt,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector()
-			throws Exception {
-		/*
-		 * Join on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.join(ds2)
-						.where(new KeySelector1())
-						.equalTo(0)
-						.with(new CustT3Join());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hi\n" +
-				"Hello,Hello\n" +
-				"Hello world,Hello\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	public static class KeySelector1 implements KeySelector<CustomType, Integer> {
-		@Override
-		public Integer getKey(CustomType value) {
-			return value.myInt;
-		}
-	}
-
-	@Test
-	public void testProjectOnATuple1Input() throws Exception {
-		/*
-		 * Project join on a tuple input 1
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple6<String, Long, String, Integer, Long, Long>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(1)
-						.projectFirst(2, 1)
-						.projectSecond(3)
-						.projectFirst(0)
-						.projectSecond(4, 1);
-
-		List<Tuple6<String, Long, String, Integer, Long, Long>> result = joinDs.collect();
-
-		String expected = "Hi,1,Hallo,1,1,1\n" +
-				"Hello,2,Hallo Welt,2,2,2\n" +
-				"Hello world,2,Hallo Welt,3,2,2\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testProjectJoinOnATuple2Input() throws Exception {
-		/*
-		 * Project join on a tuple input 2
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple6<String, String, Long, Long, Long, Integer>> joinDs =
-				ds1.join(ds2)
-						.where(1)
-						.equalTo(1)
-						.projectSecond(3)
-						.projectFirst(2, 1)
-						.projectSecond(4, 1)
-						.projectFirst(0);
-
-		List<Tuple6<String, String, Long, Long, Long, Integer>> result = joinDs.collect();
-
-		String expected = "Hallo,Hi,1,1,1,1\n" +
-				"Hallo Welt,Hello,2,2,2,2\n" +
-				"Hallo Welt,Hello world,2,2,2,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
-			throws Exception {
-		/*
-		 * Join on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.join(ds2)
-						.where(1).equalTo(new KeySelector2())
-						.with(new T3CustJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hello\n" +
-				"Hello,Hello world\n" +
-				"Hello world,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<CustomType, Long> {
-		@Override
-		public Long getKey(CustomType value) {
-			return value.myLong;
-		}
-	}
-
-	@Test
-	public void testDefaultJoinOnTwoCustomTypeInputsWithKeyExtractors() throws Exception {
-		/*
-		 * (Default) Join on two custom type inputs with key extractors
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-
-		DataSet<Tuple2<CustomType, CustomType>> joinDs =
-				ds1.join(ds2)
-						.where(new KeySelector5())
-						.equalTo(new KeySelector6());
-
-		List<Tuple2<CustomType, CustomType>> result = joinDs.collect();
-
-		String expected = "1,0,Hi,1,0,Hi\n" +
-				"2,1,Hello,2,1,Hello\n" +
-				"2,1,Hello,2,2,Hello world\n" +
-				"2,2,Hello world,2,1,Hello\n" +
-				"2,2,Hello world,2,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsClosureCleaner() throws Exception {
-		/*
-		 * (Default) Join on two custom type inputs with key extractors, implemented as inner classes to test closure
-		 * cleaning
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-
-		DataSet<Tuple2<CustomType, CustomType>> joinDs =
-				ds1.join(ds2)
-						.where(new KeySelector<CustomType, Integer>() {
-							@Override
-							public Integer getKey(CustomType value) {
-								return value.myInt;
-							}
-						})
-						.equalTo(new KeySelector<CustomType, Integer>() {
-
-							@Override
-							public Integer getKey(CustomType value) throws Exception {
-								return value.myInt;
-							}
-						});
-
-		List<Tuple2<CustomType, CustomType>> result = joinDs.collect();
-
-		String expected = "1,0,Hi,1,0,Hi\n" +
-				"2,1,Hello,2,1,Hello\n" +
-				"2,1,Hello,2,2,Hello world\n" +
-				"2,2,Hello world,2,1,Hello\n" +
-				"2,2,Hello world,2,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsDisabledClosureCleaner() throws Exception {
-		/*
-		 * (Default) Join on two custom type inputs with key extractors, check if disableing closure cleaning works
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().disableClosureCleaner();
-
-		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		boolean correctExceptionTriggered = false;
-		try {
-			DataSet<Tuple2<CustomType, CustomType>> joinDs =
-					ds1.join(ds2)
-							.where(new KeySelector<CustomType, Integer>() {
-								@Override
-								public Integer getKey(CustomType value) {
-									return value.myInt;
-								}
-							})
-							.equalTo(new KeySelector<CustomType, Integer>() {
-
-								@Override
-								public Integer getKey(CustomType value) throws Exception {
-									return value.myInt;
-								}
-							});
-		} catch (InvalidProgramException ex) {
-			correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException);
-		}
-		Assert.assertTrue(correctExceptionTriggered);
-	}
-
-	public static class KeySelector5 implements KeySelector<CustomType, Integer> {
-		@Override
-		public Integer getKey(CustomType value) {
-			return value.myInt;
-		}
-	}
-
-	public static class KeySelector6 implements KeySelector<CustomType, Integer> {
-		@Override
-		public Integer getKey(CustomType value) {
-			return value.myInt;
-		}
-	}
-
-	@Test
-	public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception {
-		/*
-		 * UDF Join on tuples with tuple-returning key selectors
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<String, String>> joinDs =
-				ds1.join(ds2)
-						.where(new KeySelector3())
-						.equalTo(new KeySelector4())
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt wie gehts?\n" +
-				"Hello world,ABC\n" +
-				"I am fine.,HIJ\n" +
-				"I am fine.,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector3 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f1);
-		}
-	}
-
-	public static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f4);
-		}
-	}
-
-	@Test
-	public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception {
-		/*
-		 * Join nested pojo against tuple (selected using a string)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.join(ds2).where("nestedPojo.longNumber").equalTo("f6");
-
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exception {
-		/*
-		 * Join nested pojo against tuple (selected as an integer)
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.join(ds2).where("nestedPojo.longNumber").equalTo(6); // <--- difference!
-
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exception {
-		/*
-		 * selecting multiple fields using expression language
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.join(ds2).where("nestedPojo.longNumber", "number", "str").equalTo("f6", "f0", "f1");
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedIntoTuple() throws Exception {
-		/*
-		 * nested into tuple
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.join(ds2).where("nestedPojo.longNumber", "number", "nestedTupleWithCustom.f0").equalTo("f6", "f0", "f2");
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedIntoTupleIntoPojo() throws Exception {
-		/*
-		 * nested into tuple into pojo
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
-				ds1.join(ds2).where("nestedTupleWithCustom.f0", "nestedTupleWithCustom.f1.myInt", "nestedTupleWithCustom.f1.myLong").equalTo("f2", "f3", "f4");
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
-				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
-				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNonPojoToVerifyFullTupleKeys() throws Exception {
-		/*
-		 * Non-POJO test to verify that full-tuple keys are working.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
-				ds1.join(ds2).where(0).equalTo("f0.f0", "f0.f1"); // key is now Tuple2<Integer, Integer>
-
-		env.setParallelism(1);
-		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
-
-		String expected = "((1,1),one),((1,1),one)\n" +
-				"((2,2),two),((2,2),two)\n" +
-				"((3,3),three),((3,3),three)\n";
-
-		compareResultAsTuples(result, expected);
-
-	}
-
-	@Test
-	public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception {
-		/*
-		 * Non-POJO test to verify "nested" tuple-element selection.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
-		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
-				ds1.join(ds2).where("f0.f0").equalTo("f0.f0"); // key is now Integer from Tuple2<Integer, Integer>
-
-		env.setParallelism(1);
-		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
-
-		String expected = "((1,1),one),((1,1),one)\n" +
-				"((2,2),two),((2,2),two)\n" +
-				"((3,3),three),((3,3),three)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testFullPojoWithFullTuple() throws Exception {
-		/*
-		 * full pojo with full tuple
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env);
-		DataSet<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> joinDs =
-				ds1.join(ds2).where("*").equalTo("*");
-
-		env.setParallelism(1);
-		List<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> result = joinDs.collect();
-
-		String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
-				"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +
-				"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreaterThanZero()
-			throws Exception {
-		/*
-		 * Non-POJO test to verify "nested" tuple-element selection with the first key field greater than 0.
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>> ds2 = ds1.join(ds1).where(0).equalTo(0);
-		DataSet<Tuple2<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>, Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>>> joinDs =
-				ds2.join(ds2).where("f1.f0").equalTo("f0.f0");
-
-		env.setParallelism(1);
-		List<Tuple2<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>, Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>>> result = joinDs.collect();
-
-		String expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" +
-				"((2,2,Hello),(2,2,Hello)),((2,2,Hello),(2,2,Hello))\n" +
-				"((3,2,Hello world),(3,2,Hello world)),((3,2,Hello world),(3,2,Hello world))\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithAtomicType1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Integer> ds2 = env.fromElements(1, 2);
-
-		DataSet<Tuple2<Tuple3<Integer, Long, String>, Integer>> joinDs = ds1.join(ds2).where(0).equalTo("*");
-
-		List<Tuple2<Tuple3<Integer, Long, String>, Integer>> result = joinDs.collect();
-
-		String expected = "(1,1,Hi),1\n" +
-				"(2,2,Hello),2";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithAtomicType2() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ds1 = env.fromElements(1, 2);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-
-		DataSet<Tuple2<Integer, Tuple3<Integer, Long, String>>> joinDs = ds1.join(ds2).where("*").equalTo(0);
-
-		List<Tuple2<Integer, Tuple3<Integer, Long, String>>> result = joinDs.collect();
-
-		String expected = "1,(1,1,Hi)\n" +
-				"2,(2,2,Hello)";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testJoinWithRangePartitioning() throws Exception {
-		/*
-		 * Test Join on tuples with multiple key field positions and same customized distribution
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		env.setParallelism(4);
-		TestDistribution testDis = new TestDistribution();
-		DataSet<Tuple2<String, String>> joinDs =
-				DataSetUtils.partitionByRange(ds1, testDis, 0, 1)
-						.join(DataSetUtils.partitionByRange(ds2, testDis, 0, 4))
-						.where(0, 1)
-						.equalTo(0, 4)
-						.with(new T3T5FlatJoin());
-
-		List<Tuple2<String, String>> result = joinDs.collect();
-
-		String expected = "Hi,Hallo\n" +
-				"Hello,Hallo Welt\n" +
-				"Hello world,Hallo Welt wie gehts?\n" +
-				"Hello world,ABC\n" +
-				"I am fine.,HIJ\n" +
-				"I am fine.,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class T3T5FlatJoin implements FlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<String, String>> {
-
-		@Override
-		public void join(Tuple3<Integer, Long, String> first,
-		                 Tuple5<Integer, Long, Integer, String, Long> second,
-		                 Collector<Tuple2<String, String>> out) {
-
-			out.collect(new Tuple2<String, String>(first.f2, second.f3));
-		}
-
-	}
-
-	public static class LeftReturningJoin implements JoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>> {
-
-		@Override
-		public Tuple3<Integer, Long, String> join(Tuple3<Integer, Long, String> first,
-		                                          Tuple5<Integer, Long, Integer, String, Long> second) {
-
-			return first;
-		}
-	}
-
-	public static class RightReturningJoin implements JoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
-
-		@Override
-		public Tuple5<Integer, Long, Integer, String, Long> join(Tuple3<Integer, Long, String> first,
-		                                                         Tuple5<Integer, Long, Integer, String, Long> second) {
-
-			return second;
-		}
-	}
-
-	public static class T3T5BCJoin extends RichFlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<String, String, Integer>> {
-
-		private int broadcast;
-
-		@Override
-		public void open(Configuration config) {
-
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for (Integer i : ints) {
-				sum += i;
-			}
-			broadcast = sum;
-
-		}
-
-		/*
-		@Override
-		public Tuple3<String, String, Integer> join(
-				Tuple3<Integer, Long, String> first,
-				Tuple5<Integer, Long, Integer, String, Long> second) {
-
-			return new Tuple3<String, String, Integer>(first.f2, second.f3, broadcast);
-		}
-		 */
-
-		@Override
-		public void join(Tuple3<Integer, Long, String> first, Tuple5<Integer, Long, Integer, String, Long> second, Collector<Tuple3<String, String, Integer>> out) throws Exception {
-			out.collect(new Tuple3<String, String, Integer>(first.f2, second.f3, broadcast));
-		}
-	}
-
-	public static class T3CustJoin implements JoinFunction<Tuple3<Integer, Long, String>, CustomType, Tuple2<String, String>> {
-
-		@Override
-		public Tuple2<String, String> join(Tuple3<Integer, Long, String> first,
-		                                   CustomType second) {
-
-			return new Tuple2<String, String>(first.f2, second.myString);
-		}
-	}
-
-	public static class CustT3Join implements JoinFunction<CustomType, Tuple3<Integer, Long, String>, Tuple2<String, String>> {
-
-		@Override
-		public Tuple2<String, String> join(CustomType first, Tuple3<Integer, Long, String> second) {
-
-			return new Tuple2<String, String>(first.myString, second.f2);
-		}
-	}
-
-	public static class TestDistribution implements DataDistribution {
-		public Object boundaries[][] = new Object[][]{
-				new Object[]{2, 2L},
-				new Object[]{5, 4L},
-				new Object[]{10, 12L},
-				new Object[]{21, 6L}
-		};
-
-		public TestDistribution() {
-		}
-
-		@Override
-		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
-			return boundaries[bucketNum];
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 2;
-		}
-
-		@Override
-		public TypeInformation[] getKeyTypes() {
-			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO};
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-
-		}
-
-		@Override
-		public void read(DataInputView in) throws IOException {
-
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj instanceof TestDistribution;
-		}
-	}
-}


[06/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
new file mode 100644
index 0000000..2d6897b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java
@@ -0,0 +1,515 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.PojoWithDateAndEnum;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Integration tests for {@link ReduceFunction} and {@link RichReduceFunction}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class ReduceITCase extends MultipleProgramsTestBase {
+
+	public ReduceITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testReduceOnTuplesWithKeyFieldSelector() throws Exception {
+		/*
+		 * Reduce on tuples with key field selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).reduce(new Tuple3Reduce("B-)"));
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"5,2,B-)\n" +
+				"15,3,B-)\n" +
+				"34,4,B-)\n" +
+				"65,5,B-)\n" +
+				"111,6,B-)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception{
+		/*
+		 * Reduce on tuples with multiple key field selectors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
+				groupBy(4, 0).reduce(new Tuple5Reduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
+				.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"3,9,0,P-),2\n" +
+				"3,6,5,BCD,3\n" +
+				"4,17,0,P-),1\n" +
+				"4,17,0,P-),2\n" +
+				"5,11,10,GHI,1\n" +
+				"5,29,0,P-),2\n" +
+				"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testReduceOnTuplesWithKeyExtractor() throws Exception {
+		/*
+		 * Reduce on tuples with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(new KeySelector1()).reduce(new Tuple3Reduce("B-)"));
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"5,2,B-)\n" +
+				"15,3,B-)\n" +
+				"34,4,B-)\n" +
+				"65,5,B-)\n" +
+				"111,6,B-)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector1 implements KeySelector<Tuple3<Integer, Long, String>, Long> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Long getKey(Tuple3<Integer, Long, String> in) {
+			return in.f1;
+		}
+	}
+
+	@Test
+	public void testReduceOnCustomTypeWithKeyExtractor() throws Exception {
+		/*
+		 * Reduce on custom type with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds.
+				groupBy(new KeySelector2()).reduce(new CustomTypeReduce());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "1,0,Hi\n" +
+				"2,3,Hello!\n" +
+				"3,12,Hello!\n" +
+				"4,30,Hello!\n" +
+				"5,60,Hello!\n" +
+				"6,105,Hello!\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testAllReduceForTuple() throws Exception {
+		/*
+		 * All-reduce for tuple
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				reduce(new AllAddingTuple3Reduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "231,91,Hello World\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testAllReduceForCustomTypes() throws Exception {
+		/*
+		 * All-reduce for custom types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds.
+				reduce(new AllAddingCustomTypeReduce());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "91,210,Hello!";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testReduceWithBroadcastSet() throws Exception {
+		/*
+		 * Reduce with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).reduce(new BCTuple3Reduce()).withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"5,2,55\n" +
+				"15,3,55\n" +
+				"34,4,55\n" +
+				"65,5,55\n" +
+				"111,6,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testReduceATupleReturningKeySelector() throws Exception {
+		/*
+		 * Reduce with a Tuple-returning KeySelector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> reduceDs = ds
+				.groupBy(new KeySelector3()).reduce(new Tuple5Reduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
+				.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"3,9,0,P-),2\n" +
+				"3,6,5,BCD,3\n" +
+				"4,17,0,P-),1\n" +
+				"4,17,0,P-),2\n" +
+				"5,11,10,GHI,1\n" +
+				"5,29,0,P-),2\n" +
+				"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector3 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testReduceOnTupleWithMultipleKeyExpressions() throws Exception {
+		/*
+		 * Case 2 with String-based field expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds
+				.groupBy("f4", "f0").reduce(new Tuple5Reduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
+				.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"2,3,2,Hallo Welt wie,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"3,9,0,P-),2\n" +
+				"3,6,5,BCD,3\n" +
+				"4,17,0,P-),1\n" +
+				"4,17,0,P-),2\n" +
+				"5,11,10,GHI,1\n" +
+				"5,29,0,P-),2\n" +
+				"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testReduceOnTupleWithMultipleKeyExpressionsWithHashHint() throws Exception {
+		/*
+		 * Case 2 with String-based field expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds
+			.groupBy("f4", "f0").reduce(new Tuple5Reduce()).setCombineHint(CombineHint.HASH);
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
+			.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+			"2,3,2,Hallo Welt wie,1\n" +
+			"2,2,1,Hallo Welt,2\n" +
+			"3,9,0,P-),2\n" +
+			"3,6,5,BCD,3\n" +
+			"4,17,0,P-),1\n" +
+			"4,17,0,P-),2\n" +
+			"5,11,10,GHI,1\n" +
+			"5,29,0,P-),2\n" +
+			"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testSupportForDataAndEnumSerialization() throws Exception {
+		/**
+		 * Test support for Date and enum serialization
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<PojoWithDateAndEnum> ds = env.generateSequence(0, 2).map(new Mapper1());
+		ds = ds.union(CollectionDataSets.getPojoWithDateAndEnum(env));
+
+		DataSet<String> res = ds.groupBy("group").reduceGroup(new GroupReducer1());
+
+		List<String> result = res.collect();
+
+		String expected = "ok\nok";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper1 implements MapFunction<Long, PojoWithDateAndEnum> {
+		@Override
+		public PojoWithDateAndEnum map(Long value) throws Exception {
+			int l = value.intValue();
+			switch (l) {
+				case 0:
+					PojoWithDateAndEnum one = new PojoWithDateAndEnum();
+					one.group = "a";
+					one.date = new Date(666);
+					one.cat = CollectionDataSets.Category.CAT_A;
+					return one;
+				case 1:
+					PojoWithDateAndEnum two = new PojoWithDateAndEnum();
+					two.group = "a";
+					two.date = new Date(666);
+					two.cat = CollectionDataSets.Category.CAT_A;
+					return two;
+				case 2:
+					PojoWithDateAndEnum three = new PojoWithDateAndEnum();
+					three.group = "b";
+					three.date = new Date(666);
+					three.cat = CollectionDataSets.Category.CAT_B;
+					return three;
+			}
+			throw new RuntimeException("Unexpected value for l=" + l);
+		}
+	}
+
+	private static class GroupReducer1 implements GroupReduceFunction<CollectionDataSets.PojoWithDateAndEnum, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<PojoWithDateAndEnum> values,
+				Collector<String> out) throws Exception {
+			for (PojoWithDateAndEnum val : values) {
+				if (val.cat == CollectionDataSets.Category.CAT_A) {
+					Assert.assertEquals("a", val.group);
+				} else if (val.cat == CollectionDataSets.Category.CAT_B) {
+					Assert.assertEquals("b", val.group);
+				} else {
+					Assert.fail("error. Cat = " + val.cat);
+				}
+				Assert.assertEquals(666, val.date.getTime());
+			}
+			out.collect("ok");
+		}
+	}
+
+	private static class Tuple3Reduce implements ReduceFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
+		private final String f2Replace;
+
+		public Tuple3Reduce() {
+			this.f2Replace = null;
+		}
+
+		public Tuple3Reduce(String f2Replace) {
+			this.f2Replace = f2Replace;
+		}
+
+		@Override
+		public Tuple3<Integer, Long, String> reduce(
+				Tuple3<Integer, Long, String> in1,
+				Tuple3<Integer, Long, String> in2) throws Exception {
+
+			if (f2Replace == null) {
+				out.setFields(in1.f0 + in2.f0, in1.f1, in1.f2);
+			} else {
+				out.setFields(in1.f0 + in2.f0, in1.f1, this.f2Replace);
+			}
+			return out;
+		}
+	}
+
+	private static class Tuple5Reduce implements ReduceFunction<Tuple5<Integer, Long, Integer, String, Long>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple5<Integer, Long, Integer, String, Long> out = new Tuple5<Integer, Long, Integer, String, Long>();
+
+		@Override
+		public Tuple5<Integer, Long, Integer, String, Long> reduce(
+				Tuple5<Integer, Long, Integer, String, Long> in1,
+				Tuple5<Integer, Long, Integer, String, Long> in2)
+						throws Exception {
+
+			out.setFields(in1.f0, in1.f1 + in2.f1, 0, "P-)", in1.f4);
+			return out;
+		}
+	}
+
+	private static class CustomTypeReduce implements ReduceFunction<CustomType> {
+		private static final long serialVersionUID = 1L;
+		private final CustomType out = new CustomType();
+
+		@Override
+		public CustomType reduce(CustomType in1, CustomType in2)
+				throws Exception {
+
+			out.myInt = in1.myInt;
+			out.myLong = in1.myLong + in2.myLong;
+			out.myString = "Hello!";
+			return out;
+		}
+	}
+
+	private static class AllAddingTuple3Reduce implements ReduceFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
+
+		@Override
+		public Tuple3<Integer, Long, String> reduce(
+				Tuple3<Integer, Long, String> in1,
+				Tuple3<Integer, Long, String> in2) throws Exception {
+
+			out.setFields(in1.f0 + in2.f0, in1.f1 + in2.f1, "Hello World");
+			return out;
+		}
+	}
+
+	private static class AllAddingCustomTypeReduce implements ReduceFunction<CustomType> {
+		private static final long serialVersionUID = 1L;
+		private final CustomType out = new CustomType();
+
+		@Override
+		public CustomType reduce(CustomType in1, CustomType in2)
+				throws Exception {
+
+			out.myInt = in1.myInt + in2.myInt;
+			out.myLong = in1.myLong + in2.myLong;
+			out.myString = "Hello!";
+			return out;
+		}
+	}
+
+	private static class BCTuple3Reduce extends RichReduceFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
+		private String f2Replace = "";
+
+		@Override
+		public void open(Configuration config) {
+
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			f2Replace = sum + "";
+
+		}
+
+		@Override
+		public Tuple3<Integer, Long, String> reduce(
+				Tuple3<Integer, Long, String> in1,
+				Tuple3<Integer, Long, String> in2) throws Exception {
+
+			out.setFields(in1.f0 + in2.f0, in1.f1, this.f2Replace);
+			return out;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java
new file mode 100644
index 0000000..c6d340a
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceWithCombinerITCase.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.CombineFunction;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.UnsortedGrouping;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Integration tests for {@link GroupCombineFunction}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class ReduceWithCombinerITCase extends MultipleProgramsTestBase {
+
+	public ReduceWithCombinerITCase(TestExecutionMode mode) {
+		super(TestExecutionMode.CLUSTER);
+	}
+
+	@Test
+	public void testReduceOnNonKeyedDataset() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple2<Integer, Boolean>> input = createNonKeyedInput(env);
+		List<Tuple2<Integer, Boolean>> actual = input.reduceGroup(new NonKeyedCombReducer()).collect();
+		String expected = "10,true\n";
+
+		compareResultAsTuples(actual, expected);
+	}
+
+	@Test
+	public void testForkingReduceOnNonKeyedDataset() throws Exception {
+
+		// set up the execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple2<Integer, Boolean>> input = createNonKeyedInput(env);
+
+		DataSet<Tuple2<Integer, Boolean>> r1 = input.reduceGroup(new NonKeyedCombReducer());
+		DataSet<Tuple2<Integer, Boolean>> r2 = input.reduceGroup(new NonKeyedGroupCombReducer());
+
+		List<Tuple2<Integer, Boolean>> actual = r1.union(r2).collect();
+		String expected = "10,true\n10,true\n";
+		compareResultAsTuples(actual, expected);
+	}
+
+	@Test
+	public void testReduceOnKeyedDataset() throws Exception {
+
+		// set up the execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
+		List<Tuple3<String, Integer, Boolean>> actual = input.groupBy(0).reduceGroup(new KeyedCombReducer()).collect();
+		String expected = "k1,6,true\nk2,4,true\n";
+
+		compareResultAsTuples(actual, expected);
+	}
+
+	@Test
+	public void testReduceOnKeyedDatasetWithSelector() throws Exception {
+
+		// set up the execution environment
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
+
+		List<Tuple3<String, Integer, Boolean>> actual = input
+			.groupBy(new KeySelectorX())
+			.reduceGroup(new KeyedCombReducer())
+			.collect();
+		String expected = "k1,6,true\nk2,4,true\n";
+
+		compareResultAsTuples(actual, expected);
+	}
+
+	@Test
+	public void testForkingReduceOnKeyedDataset() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
+
+		UnsortedGrouping<Tuple3<String, Integer, Boolean>> counts = input.groupBy(0);
+
+		DataSet<Tuple3<String, Integer, Boolean>> r1 = counts.reduceGroup(new KeyedCombReducer());
+		DataSet<Tuple3<String, Integer, Boolean>> r2 = counts.reduceGroup(new KeyedGroupCombReducer());
+
+		List<Tuple3<String, Integer, Boolean>> actual = r1.union(r2).collect();
+		String expected = "k1,6,true\n" +
+			"k2,4,true\n" +
+			"k1,6,true\n" +
+			"k2,4,true\n";
+		compareResultAsTuples(actual, expected);
+	}
+
+	@Test
+	public void testForkingReduceOnKeyedDatasetWithSelection() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		// creates the input data and distributes them evenly among the available downstream tasks
+		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
+
+		UnsortedGrouping<Tuple3<String, Integer, Boolean>> counts = input.groupBy(new KeySelectorX());
+
+		DataSet<Tuple3<String, Integer, Boolean>> r1 = counts.reduceGroup(new KeyedCombReducer());
+		DataSet<Tuple3<String, Integer, Boolean>> r2 = counts.reduceGroup(new KeyedGroupCombReducer());
+
+		List<Tuple3<String, Integer, Boolean>> actual = r1.union(r2).collect();
+		String expected = "k1,6,true\n" +
+			"k2,4,true\n" +
+			"k1,6,true\n" +
+			"k2,4,true\n";
+
+		compareResultAsTuples(actual, expected);
+	}
+
+	private DataSet<Tuple2<Integer, Boolean>> createNonKeyedInput(ExecutionEnvironment env) {
+		return env.fromCollection(Arrays.asList(
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false),
+			new Tuple2<>(1, false))
+		).rebalance();
+	}
+
+	private static class NonKeyedCombReducer implements CombineFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>>,
+		GroupReduceFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>> {
+
+		@Override
+		public Tuple2<Integer, Boolean> combine(Iterable<Tuple2<Integer, Boolean>> values) throws Exception {
+			int sum = 0;
+			boolean flag = true;
+
+			for (Tuple2<Integer, Boolean> tuple : values) {
+				sum += tuple.f0;
+				flag &= !tuple.f1;
+
+			}
+			return new Tuple2<>(sum, flag);
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
+			int sum = 0;
+			boolean flag = true;
+			for (Tuple2<Integer, Boolean> tuple : values) {
+				sum += tuple.f0;
+				flag &= tuple.f1;
+			}
+			out.collect(new Tuple2<>(sum, flag));
+		}
+	}
+
+	private static class NonKeyedGroupCombReducer implements GroupCombineFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>>,
+		GroupReduceFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>> {
+
+		@Override
+		public void reduce(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
+			int sum = 0;
+			boolean flag = true;
+			for (Tuple2<Integer, Boolean> tuple : values) {
+				sum += tuple.f0;
+				flag &= tuple.f1;
+			}
+			out.collect(new Tuple2<>(sum, flag));
+		}
+
+		@Override
+		public void combine(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
+			int sum = 0;
+			boolean flag = true;
+			for (Tuple2<Integer, Boolean> tuple : values) {
+				sum += tuple.f0;
+				flag &= !tuple.f1;
+			}
+			out.collect(new Tuple2<>(sum, flag));
+		}
+	}
+
+	private DataSet<Tuple3<String, Integer, Boolean>> createKeyedInput(ExecutionEnvironment env) {
+		return env.fromCollection(Arrays.asList(
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k2", 1, false),
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k2", 1, false),
+			new Tuple3<>("k2", 1, false),
+			new Tuple3<>("k1", 1, false),
+			new Tuple3<>("k2", 1, false))
+		).rebalance();
+	}
+
+	private static class KeySelectorX implements KeySelector<Tuple3<String, Integer, Boolean>, String> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public String getKey(Tuple3<String, Integer, Boolean> in) {
+			return in.f0;
+		}
+	}
+
+	private class KeyedCombReducer implements CombineFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>>,
+		GroupReduceFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>> {
+
+		@Override
+		public Tuple3<String, Integer, Boolean> combine(Iterable<Tuple3<String, Integer, Boolean>> values) throws Exception {
+			String key = null;
+			int sum = 0;
+			boolean flag = true;
+
+			for (Tuple3<String, Integer, Boolean> tuple : values) {
+				key = (key == null) ? tuple.f0 : key;
+				sum += tuple.f1;
+				flag &= !tuple.f2;
+			}
+			return new Tuple3<>(key, sum, flag);
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
+			String key = null;
+			int sum = 0;
+			boolean flag = true;
+
+			for (Tuple3<String, Integer, Boolean> tuple : values) {
+				key = (key == null) ? tuple.f0 : key;
+				sum += tuple.f1;
+				flag &= tuple.f2;
+			}
+			out.collect(new Tuple3<>(key, sum, flag));
+		}
+	}
+
+	private class KeyedGroupCombReducer implements GroupCombineFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>>,
+		GroupReduceFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>> {
+
+		@Override
+		public void combine(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
+			String key = null;
+			int sum = 0;
+			boolean flag = true;
+
+			for (Tuple3<String, Integer, Boolean> tuple : values) {
+				key = (key == null) ? tuple.f0 : key;
+				sum += tuple.f1;
+				flag &= !tuple.f2;
+			}
+			out.collect(new Tuple3<>(key, sum, flag));
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
+			String key = null;
+			int sum = 0;
+			boolean flag = true;
+
+			for (Tuple3<String, Integer, Boolean> tuple : values) {
+				key = (key == null) ? tuple.f0 : key;
+				sum += tuple.f1;
+				flag &= tuple.f2;
+			}
+			out.collect(new Tuple3<>(key, sum, flag));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java
new file mode 100644
index 0000000..36eded6
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.io.GenericInputFormat;
+import org.apache.flink.api.common.operators.util.TestNonRichInputFormat;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.AkkaOptions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.runtime.minicluster.StandaloneMiniCluster;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Integration tests for {@link org.apache.flink.api.java.RemoteEnvironment}.
+ */
+@SuppressWarnings("serial")
+public class RemoteEnvironmentITCase extends TestLogger {
+
+	private static final int TM_SLOTS = 4;
+
+	private static final int USER_DOP = 2;
+
+	private static final String INVALID_STARTUP_TIMEOUT = "0.001 ms";
+
+	private static final String VALID_STARTUP_TIMEOUT = "100 s";
+
+	private static Configuration configuration;
+
+	private static StandaloneMiniCluster cluster;
+
+	@BeforeClass
+	public static void setupCluster() throws Exception {
+		configuration = new Configuration();
+
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
+
+		cluster = new StandaloneMiniCluster(configuration);
+	}
+
+	@AfterClass
+	public static void tearDownCluster() throws Exception {
+		cluster.close();
+	}
+
+	/**
+	 * Ensure that that Akka configuration parameters can be set.
+	 */
+	@Test(expected = FlinkException.class)
+	public void testInvalidAkkaConfiguration() throws Throwable {
+		Configuration config = new Configuration();
+		config.setString(AkkaOptions.STARTUP_TIMEOUT, INVALID_STARTUP_TIMEOUT);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				cluster.getHostname(),
+				cluster.getPort(),
+				config
+		);
+		env.getConfig().disableSysoutLogging();
+
+		DataSet<String> result = env.createInput(new TestNonRichInputFormat());
+		result.output(new LocalCollectionOutputFormat<>(new ArrayList<String>()));
+		try {
+			env.execute();
+			Assert.fail("Program should not run successfully, cause of invalid akka settings.");
+		} catch (ProgramInvocationException ex) {
+			throw ex.getCause();
+		}
+	}
+
+	/**
+	 * Ensure that the program parallelism can be set even if the configuration is supplied.
+	 */
+	@Test
+	public void testUserSpecificParallelism() throws Exception {
+		Configuration config = new Configuration();
+		config.setString(AkkaOptions.STARTUP_TIMEOUT, VALID_STARTUP_TIMEOUT);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
+				cluster.getHostname(),
+				cluster.getPort(),
+				config
+		);
+		env.setParallelism(USER_DOP);
+		env.getConfig().disableSysoutLogging();
+
+		DataSet<Integer> result = env.createInput(new ParallelismDependentInputFormat())
+				.rebalance()
+				.mapPartition(new RichMapPartitionFunction<Integer, Integer>() {
+					@Override
+					public void mapPartition(Iterable<Integer> values, Collector<Integer> out) throws Exception {
+						out.collect(getRuntimeContext().getIndexOfThisSubtask());
+					}
+				});
+		List<Integer> resultCollection = result.collect();
+		assertEquals(USER_DOP, resultCollection.size());
+	}
+
+	private static class ParallelismDependentInputFormat extends GenericInputFormat<Integer> {
+
+		private transient boolean emitted;
+
+		@Override
+		public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
+			assertEquals(USER_DOP, numSplits);
+			return super.createInputSplits(numSplits);
+		}
+
+		@Override
+		public boolean reachedEnd() {
+			return emitted;
+		}
+
+		@Override
+		public Integer nextRecord(Integer reuse) {
+			if (emitted) {
+				return null;
+			}
+			emitted = true;
+			return 1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java
new file mode 100644
index 0000000..c023cf4
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReplicatingDataSourceITCase.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.io.ReplicatingInputFormat;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.ParallelIteratorInputFormat;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Tests for replicating DataSources.
+ */
+@RunWith(Parameterized.class)
+public class ReplicatingDataSourceITCase extends MultipleProgramsTestBase {
+
+	public ReplicatingDataSourceITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testReplicatedSourceToJoin() throws Exception {
+		/*
+		 * Test replicated source going into join
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple1<Long>> source1 = env.createInput(new ReplicatingInputFormat<Long, GenericInputSplit>
+				(new ParallelIteratorInputFormat<Long>(new NumberSequenceIterator(0L, 1000L))), BasicTypeInfo.LONG_TYPE_INFO)
+				.map(new ToTuple());
+		DataSet<Tuple1<Long>> source2 = env.generateSequence(0L, 1000L).map(new ToTuple());
+
+		DataSet<Tuple> pairs = source1.join(source2).where(0).equalTo(0)
+				.projectFirst(0)
+				.sum(0);
+
+		List<Tuple> result = pairs.collect();
+
+		String expectedResult = "(500500)";
+
+		compareResultAsText(result, expectedResult);
+	}
+
+	@Test
+	public void testReplicatedSourceToCross() throws Exception {
+		/*
+		 * Test replicated source going into cross
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple1<Long>> source1 = env.createInput(new ReplicatingInputFormat<Long, GenericInputSplit>
+				(new ParallelIteratorInputFormat<Long>(new NumberSequenceIterator(0L, 1000L))), BasicTypeInfo.LONG_TYPE_INFO)
+				.map(new ToTuple());
+		DataSet<Tuple1<Long>> source2 = env.generateSequence(0L, 1000L).map(new ToTuple());
+
+		DataSet<Tuple1<Long>> pairs = source1.cross(source2)
+				.filter(new FilterFunction<Tuple2<Tuple1<Long>, Tuple1<Long>>>() {
+					@Override
+					public boolean filter(Tuple2<Tuple1<Long>, Tuple1<Long>> value) throws Exception {
+						return value.f0.f0.equals(value.f1.f0);
+					}
+				})
+				.map(new MapFunction<Tuple2<Tuple1<Long>, Tuple1<Long>>, Tuple1<Long>>() {
+					@Override
+					public Tuple1<Long> map(Tuple2<Tuple1<Long>, Tuple1<Long>> value) throws Exception {
+						return value.f0;
+					}
+				})
+				.sum(0);
+
+		List<Tuple1<Long>> result = pairs.collect();
+
+		String expectedResult = "(500500)";
+
+		compareResultAsText(result, expectedResult);
+	}
+
+	private static class ToTuple implements MapFunction<Long, Tuple1<Long>> {
+
+		@Override
+		public Tuple1<Long> map(Long value) throws Exception {
+			return new Tuple1<Long>(value);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java
new file mode 100644
index 0000000..c0cc62a
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/SampleITCase.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.MapPartitionOperator;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.utils.DataSetUtils;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Integration tests for {@link DataSetUtils#sample}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class SampleITCase extends MultipleProgramsTestBase {
+
+	private static final Random RNG = new Random();
+
+	public SampleITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Before
+	public void initiate() {
+		ExecutionEnvironment.getExecutionEnvironment().setParallelism(5);
+	}
+
+	@Test
+	public void testSamplerWithFractionWithoutReplacement() throws Exception {
+		verifySamplerWithFractionWithoutReplacement(0d);
+		verifySamplerWithFractionWithoutReplacement(0.2d);
+		verifySamplerWithFractionWithoutReplacement(1.0d);
+	}
+
+	@Test
+	public void testSamplerWithFractionWithReplacement() throws Exception {
+		verifySamplerWithFractionWithReplacement(0d);
+		verifySamplerWithFractionWithReplacement(0.2d);
+		verifySamplerWithFractionWithReplacement(1.0d);
+		verifySamplerWithFractionWithReplacement(2.0d);
+	}
+
+	@Test
+	public void testSamplerWithSizeWithoutReplacement() throws Exception {
+		verifySamplerWithFixedSizeWithoutReplacement(0);
+		verifySamplerWithFixedSizeWithoutReplacement(2);
+		verifySamplerWithFixedSizeWithoutReplacement(21);
+	}
+
+	@Test
+	public void testSamplerWithSizeWithReplacement() throws Exception {
+		verifySamplerWithFixedSizeWithReplacement(0);
+		verifySamplerWithFixedSizeWithReplacement(2);
+		verifySamplerWithFixedSizeWithReplacement(21);
+	}
+
+	private void verifySamplerWithFractionWithoutReplacement(double fraction) throws Exception {
+		verifySamplerWithFractionWithoutReplacement(fraction, RNG.nextLong());
+	}
+
+	private void verifySamplerWithFractionWithoutReplacement(double fraction, long seed) throws Exception {
+		verifySamplerWithFraction(false, fraction, seed);
+	}
+
+	private void verifySamplerWithFractionWithReplacement(double fraction) throws Exception {
+		verifySamplerWithFractionWithReplacement(fraction, RNG.nextLong());
+	}
+
+	private void verifySamplerWithFractionWithReplacement(double fraction, long seed) throws Exception {
+		verifySamplerWithFraction(true, fraction, seed);
+	}
+
+	private void verifySamplerWithFraction(boolean withReplacement, double fraction, long seed) throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		FlatMapOperator<Tuple3<Integer, Long, String>, String> ds = getSourceDataSet(env);
+		MapPartitionOperator<String, String> sampled = DataSetUtils.sample(ds, withReplacement, fraction, seed);
+		List<String> result = sampled.collect();
+		containsResultAsText(result, getSourceStrings());
+	}
+
+	private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples) throws Exception {
+		verifySamplerWithFixedSizeWithoutReplacement(numSamples, RNG.nextLong());
+	}
+
+	private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples, long seed) throws Exception {
+		verifySamplerWithFixedSize(false, numSamples, seed);
+	}
+
+	private void verifySamplerWithFixedSizeWithReplacement(int numSamples) throws Exception {
+		verifySamplerWithFixedSizeWithReplacement(numSamples, RNG.nextLong());
+	}
+
+	private void verifySamplerWithFixedSizeWithReplacement(int numSamples, long seed) throws Exception {
+		verifySamplerWithFixedSize(true, numSamples, seed);
+	}
+
+	private void verifySamplerWithFixedSize(boolean withReplacement, int numSamples, long seed) throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		FlatMapOperator<Tuple3<Integer, Long, String>, String> ds = getSourceDataSet(env);
+		DataSet<String> sampled = DataSetUtils.sampleWithSize(ds, withReplacement, numSamples, seed);
+		List<String> result = sampled.collect();
+		assertEquals(numSamples, result.size());
+		containsResultAsText(result, getSourceStrings());
+	}
+
+	private FlatMapOperator<Tuple3<Integer, Long, String>, String> getSourceDataSet(ExecutionEnvironment env) {
+		return CollectionDataSets.get3TupleDataSet(env).flatMap(
+			new FlatMapFunction<Tuple3<Integer, Long, String>, String>() {
+				@Override
+				public void flatMap(Tuple3<Integer, Long, String> value, Collector<String> out) throws Exception {
+					out.collect(value.f2);
+				}
+			});
+	}
+
+	private String getSourceStrings() {
+		return "Hi\n" +
+			"Hello\n" +
+			"Hello world\n" +
+			"Hello world, how are you?\n" +
+			"I am fine.\n" +
+			"Luke Skywalker\n" +
+			"Comment#1\n" +
+			"Comment#2\n" +
+			"Comment#3\n" +
+			"Comment#4\n" +
+			"Comment#5\n" +
+			"Comment#6\n" +
+			"Comment#7\n" +
+			"Comment#8\n" +
+			"Comment#9\n" +
+			"Comment#10\n" +
+			"Comment#11\n" +
+			"Comment#12\n" +
+			"Comment#13\n" +
+			"Comment#14\n" +
+			"Comment#15\n";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java
new file mode 100644
index 0000000..a44f28c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/SortPartitionITCase.java
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#sortPartition}.
+ */
+@RunWith(Parameterized.class)
+public class SortPartitionITCase extends MultipleProgramsTestBase {
+
+	public SortPartitionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testSortPartitionByKeyField() throws Exception {
+		/*
+		 * Test sort partition on key field
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
+				.sortPartition(1, Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionByTwoKeyFields() throws Exception {
+		/*
+		 * Test sort partition on two key fields
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(2);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper<Tuple5<Integer, Long, Integer, String, Long>>()).setParallelism(2) // parallelize input
+				.sortPartition(4, Order.ASCENDING)
+				.sortPartition(2, Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new Tuple5Checker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void testSortPartitionByFieldExpression() throws Exception {
+		/*
+		 * Test sort partition on field expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper()).setParallelism(4) // parallelize input
+				.sortPartition("f1", Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionByTwoFieldExpressions() throws Exception {
+		/*
+		 * Test sort partition on two field expressions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(2);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper<Tuple5<Integer, Long, Integer, String, Long>>()).setParallelism(2) // parallelize input
+				.sortPartition("f4", Order.ASCENDING)
+				.sortPartition("f2", Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new Tuple5Checker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionByNestedFieldExpression() throws Exception {
+		/*
+		 * Test sort partition on nested field expressions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper<Tuple2<Tuple2<Integer, Integer>, String>>()).setParallelism(3) // parallelize input
+				.sortPartition("f0.f1", Order.ASCENDING)
+				.sortPartition("f1", Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new NestedTupleChecker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionPojoByNestedFieldExpression() throws Exception {
+		/*
+		 * Test sort partition on field expression
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.map(new IdMapper<POJO>()).setParallelism(1) // parallelize input
+				.sortPartition("nestedTupleWithCustom.f1.myString", Order.ASCENDING)
+				.sortPartition("number", Order.DESCENDING)
+				.mapPartition(new OrderCheckMapper<>(new PojoChecker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionParallelismChange() throws Exception {
+		/*
+		 * Test sort partition with parallelism change
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(3);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+				.sortPartition(1, Order.DESCENDING).setParallelism(3) // change parallelism
+				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
+				.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionWithKeySelector1() throws Exception {
+		/*
+		 * Test sort partition on an extracted key
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+			.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
+			.sortPartition(new KeySelector<Tuple3<Integer, Long, String>, Long>() {
+				@Override
+				public Long getKey(Tuple3<Integer, Long, String> value) throws Exception {
+					return value.f1;
+				}
+			}, Order.ASCENDING)
+			.mapPartition(new OrderCheckMapper<>(new Tuple3AscendingChecker()))
+			.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testSortPartitionWithKeySelector2() throws Exception {
+		/*
+		 * Test sort partition on an extracted key
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		List<Tuple1<Boolean>> result = ds
+			.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
+			.sortPartition(new KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>>() {
+				@Override
+				public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> value) throws Exception {
+					return new Tuple2<>(value.f0, value.f1);
+				}
+			}, Order.DESCENDING)
+			.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
+			.distinct().collect();
+
+		String expected = "(true)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private interface OrderChecker<T> extends Serializable {
+		boolean inOrder(T t1, T t2);
+	}
+
+	@SuppressWarnings("serial")
+	private static class Tuple3Checker implements OrderChecker<Tuple3<Integer, Long, String>> {
+		@Override
+		public boolean inOrder(Tuple3<Integer, Long, String> t1, Tuple3<Integer, Long, String> t2) {
+			return t1.f1 >= t2.f1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static class Tuple3AscendingChecker implements OrderChecker<Tuple3<Integer, Long, String>> {
+		@Override
+		public boolean inOrder(Tuple3<Integer, Long, String> t1, Tuple3<Integer, Long, String> t2) {
+			return t1.f1 <= t2.f1;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static class Tuple5Checker implements OrderChecker<Tuple5<Integer, Long, Integer, String, Long>> {
+		@Override
+		public boolean inOrder(Tuple5<Integer, Long, Integer, String, Long> t1,
+				Tuple5<Integer, Long, Integer, String, Long> t2) {
+			return t1.f4 < t2.f4 || t1.f4.equals(t2.f4) && t1.f2 >= t2.f2;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static class NestedTupleChecker implements OrderChecker<Tuple2<Tuple2<Integer, Integer>, String>> {
+		@Override
+		public boolean inOrder(Tuple2<Tuple2<Integer, Integer>, String> t1,
+				Tuple2<Tuple2<Integer, Integer>, String> t2) {
+			return t1.f0.f1 < t2.f0.f1 ||
+					t1.f0.f1.equals(t2.f0.f1) && t1.f1.compareTo(t2.f1) >= 0;
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static class PojoChecker implements OrderChecker<POJO> {
+		@Override
+		public boolean inOrder(POJO t1, POJO t2) {
+			return t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) < 0 ||
+					t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) == 0 &&
+					t1.number >= t2.number;
+		}
+	}
+
+	@SuppressWarnings("unused, serial")
+	private static class OrderCheckMapper<T> implements MapPartitionFunction<T, Tuple1<Boolean>> {
+
+		OrderChecker<T> checker;
+
+		public OrderCheckMapper() {}
+
+		public OrderCheckMapper(OrderChecker<T> checker) {
+			this.checker = checker;
+		}
+
+		@Override
+		public void mapPartition(Iterable<T> values, Collector<Tuple1<Boolean>> out) throws Exception {
+
+			Iterator<T> it = values.iterator();
+			if (!it.hasNext()) {
+				out.collect(new Tuple1<>(true));
+			} else {
+				T last = it.next();
+
+				while (it.hasNext()) {
+					T next = it.next();
+					if (!checker.inOrder(last, next)) {
+						out.collect(new Tuple1<>(false));
+						return;
+					}
+					last = next;
+				}
+				out.collect(new Tuple1<>(true));
+			}
+		}
+	}
+
+	@SuppressWarnings("serial")
+	private static class IdMapper<T> implements MapFunction<T, T> {
+
+		@Override
+		public T map(T value) throws Exception {
+			return value;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java
new file mode 100644
index 0000000..ebec17b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/SumMinMaxITCase.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Integration tests for {@link org.apache.flink.api.scala.GroupedDataSet#min} and
+ * {@link org.apache.flink.api.scala.GroupedDataSet#max}.
+ */
+@RunWith(Parameterized.class)
+public class SumMinMaxITCase extends MultipleProgramsTestBase {
+
+	public SumMinMaxITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testSumMaxAndProject() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> sumDs = ds
+				.sum(0)
+				.andMax(1)
+				.project(0, 1);
+
+		List<Tuple2<Integer, Long>> result = sumDs.collect();
+
+		String expected = "231,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testGroupedAggregate() throws Exception {
+		/*
+		 * Grouped Aggregate
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Long, Integer>> aggregateDs = ds.groupBy(1)
+				.sum(0)
+				.project(1, 0);
+
+		List<Tuple2<Long, Integer>> result = aggregateDs.collect();
+
+		String expected = "1,1\n" +
+				"2,5\n" +
+				"3,15\n" +
+				"4,34\n" +
+				"5,65\n" +
+				"6,111\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedAggregate() throws Exception {
+		/*
+		 * Nested Aggregate
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple1<Integer>> aggregateDs = ds.groupBy(1)
+				.min(0)
+				.min(0)
+				.project(0);
+
+		List<Tuple1<Integer>> result = aggregateDs.collect();
+
+		String expected = "1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java
new file mode 100644
index 0000000..75bf8f0
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/TypeHintITCase.java
@@ -0,0 +1,330 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.JavaProgramTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Integration tests for {@link org.apache.flink.api.common.typeinfo.TypeHint}.
+ */
+@RunWith(Parameterized.class)
+public class TypeHintITCase extends JavaProgramTestBase {
+
+	private static final int NUM_PROGRAMS = 9;
+
+	private int curProgId = config.getInteger("ProgramId", -1);
+
+	public TypeHintITCase(Configuration config) {
+		super(config);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TypeHintProgs.runProgram(curProgId);
+	}
+
+	@Parameters
+	public static Collection<Object[]> getConfigurations() throws FileNotFoundException, IOException {
+
+		LinkedList<Configuration> tConfigs = new LinkedList<Configuration>();
+
+		for (int i = 1; i <= NUM_PROGRAMS; i++) {
+			Configuration config = new Configuration();
+			config.setInteger("ProgramId", i);
+			tConfigs.add(config);
+		}
+
+		return toParameterList(tConfigs);
+	}
+
+	private static class TypeHintProgs {
+
+		public static void runProgram(int progId) throws Exception {
+			switch(progId) {
+			// Test identity map with missing types and string type hint
+			case 1: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds
+						.map(new Mapper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>())
+						.returns("Tuple3<Integer, Long, String>");
+				List<Tuple3<Integer, Long, String>> result = identityMapDs.collect();
+
+				String expectedResult = "(2,2,Hello)\n" +
+						"(3,2,Hello world)\n" +
+						"(1,1,Hi)\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test identity map with missing types and type information type hint
+			case 2: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds
+						// all following generics get erased during compilation
+						.map(new Mapper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>())
+						.returns(new TupleTypeInfo<Tuple3<Integer, Long, String>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO));
+				List<Tuple3<Integer, Long, String>> result = identityMapDs
+						.collect();
+
+				String expectedResult = "(2,2,Hello)\n" +
+						"(3,2,Hello world)\n" +
+						"(1,1,Hi)\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test flat map with class type hint
+			case 3: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> identityMapDs = ds
+						.flatMap(new FlatMapper<Tuple3<Integer, Long, String>, Integer>())
+						.returns(Integer.class);
+				List<Integer> result = identityMapDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test join with type information type hint
+			case 4: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds1
+						.join(ds2)
+						.where(0)
+						.equalTo(0)
+						.with(new Joiner<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test flat join with type information type hint
+			case 5: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds1
+						.join(ds2)
+						.where(0)
+						.equalTo(0)
+						.with(new FlatJoiner<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test unsorted group reduce with type information type hint
+			case 6: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds
+						.groupBy(0)
+						.reduceGroup(new GroupReducer<Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test sorted group reduce with type information type hint
+			case 7: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds
+						.groupBy(0)
+						.sortGroup(0, Order.ASCENDING)
+						.reduceGroup(new GroupReducer<Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test combine group with type information type hint
+			case 8: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds
+						.groupBy(0)
+						.combineGroup(new GroupCombiner<Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			// Test cogroup with type information type hint
+			case 9: {
+				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+				DataSet<Integer> resultDs = ds1
+						.coGroup(ds2)
+						.where(0)
+						.equalTo(0)
+						.with(new CoGrouper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
+						.returns(BasicTypeInfo.INT_TYPE_INFO);
+				List<Integer> result = resultDs.collect();
+
+				String expectedResult = "2\n" +
+						"3\n" +
+						"1\n";
+
+				compareResultAsText(result, expectedResult);
+				break;
+			}
+			default:
+				throw new IllegalArgumentException("Invalid program id");
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private static class Mapper<T, V> implements MapFunction<T, V> {
+		private static final long serialVersionUID = 1L;
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public V map(T value) throws Exception {
+			return (V) value;
+		}
+	}
+
+	private static class FlatMapper<T, V> implements FlatMapFunction<T, V> {
+		private static final long serialVersionUID = 1L;
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		@Override
+		public void flatMap(T value, Collector<V> out) throws Exception {
+			out.collect((V) ((Tuple3) value).f0);
+		}
+	}
+
+	private static class Joiner<IN1, IN2, OUT> implements JoinFunction<IN1, IN2, OUT> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public OUT join(IN1 first, IN2 second) throws Exception {
+			return (OUT) ((Tuple3) first).f0;
+		}
+	}
+
+	private static class FlatJoiner<IN1, IN2, OUT> implements FlatJoinFunction<IN1, IN2, OUT> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void join(IN1 first, IN2 second, Collector<OUT> out) throws Exception {
+			out.collect((OUT) ((Tuple3) first).f0);
+		}
+	}
+
+	private static class GroupReducer<IN, OUT> implements GroupReduceFunction<IN, OUT> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<IN> values, Collector<OUT> out) throws Exception {
+			out.collect((OUT) ((Tuple3) values.iterator().next()).f0);
+		}
+	}
+
+	private static class GroupCombiner<IN, OUT> implements GroupCombineFunction<IN, OUT> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterable<IN> values, Collector<OUT> out) throws Exception {
+			out.collect((OUT) ((Tuple3) values.iterator().next()).f0);
+		}
+	}
+
+	private static class CoGrouper<IN1, IN2, OUT> implements CoGroupFunction<IN1, IN2, OUT> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void coGroup(Iterable<IN1> first, Iterable<IN2> second, Collector<OUT> out) throws Exception {
+			out.collect((OUT) ((Tuple3) first.iterator().next()).f0);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
new file mode 100644
index 0000000..daa9cb1
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/UnionITCase.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Integration tests for {@link DataSet#union}.
+ */
+@RunWith(Parameterized.class)
+public class UnionITCase extends MultipleProgramsTestBase {
+
+	private static final String FULL_TUPLE_3_STRING = "1,1,Hi\n" +
+			"2,2,Hello\n" +
+			"3,2,Hello world\n" +
+			"4,3,Hello world, how are you?\n" +
+			"5,3,I am fine.\n" +
+			"6,3,Luke Skywalker\n" +
+			"7,4,Comment#1\n" +
+			"8,4,Comment#2\n" +
+			"9,4,Comment#3\n" +
+			"10,4,Comment#4\n" +
+			"11,5,Comment#5\n" +
+			"12,5,Comment#6\n" +
+			"13,5,Comment#7\n" +
+			"14,5,Comment#8\n" +
+			"15,5,Comment#9\n" +
+			"16,6,Comment#10\n" +
+			"17,6,Comment#11\n" +
+			"18,6,Comment#12\n" +
+			"19,6,Comment#13\n" +
+			"20,6,Comment#14\n" +
+			"21,6,Comment#15\n";
+
+	public UnionITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testUnion2IdenticalDataSets() throws Exception {
+		/*
+		 * Union of 2 Same Data Sets
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env));
+
+		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
+
+		String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING;
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testUnion5IdenticalDataSets() throws Exception {
+		/*
+		 * Union of 5 same Data Sets, with multiple unions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env))
+				.union(CollectionDataSets.get3TupleDataSet(env))
+				.union(CollectionDataSets.get3TupleDataSet(env))
+				.union(CollectionDataSets.get3TupleDataSet(env));
+
+		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
+
+		String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING
+				+ FULL_TUPLE_3_STRING +
+				FULL_TUPLE_3_STRING +	FULL_TUPLE_3_STRING;
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testUnionWithEmptyDataSet() throws Exception {
+		/*
+		 * Test on union with empty dataset
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// Don't know how to make an empty result in an other way than filtering it
+		DataSet<Tuple3<Integer, Long, String>> empty = CollectionDataSets.get3TupleDataSet(env).
+				filter(new RichFilter1());
+
+		DataSet<Tuple3<Integer, Long, String>> unionDs = CollectionDataSets.get3TupleDataSet(env)
+				.union(empty);
+
+		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
+
+		String expected = FULL_TUPLE_3_STRING;
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichFilter1 extends RichFilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return false;
+		}
+	}
+
+}


[03/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
index 93d369a..8371230 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.PoisonPill;
-import akka.pattern.Patterns;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -35,10 +31,12 @@ import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
+import akka.pattern.Patterns;
 import org.junit.Test;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -48,13 +46,17 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
-import static org.junit.Assert.*;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import static org.junit.Assert.fail;
 
 /**
  * This test verifies the behavior of the recovery in the case when a TaskManager
  * fails (shut down) in the middle of a job execution.
  *
- * The test works with multiple in-process task managers. Initially, it starts a JobManager
+ * <p>The test works with multiple in-process task managers. Initially, it starts a JobManager
  * and two TaskManagers with 2 slots each. It submits a program with parallelism 4
  * and waits until all tasks are brought up (coordination between the test and the tasks
  * happens via shared blocking queues). It then starts another TaskManager, which is
@@ -67,7 +69,7 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 	@Test
 	public void testRestartWithFailingTaskManager() {
 
-		final int PARALLELISM = 4;
+		final int parallelism = 4;
 
 		LocalFlinkMiniCluster cluster = null;
 		ActorSystem additionalSystem = null;
@@ -75,9 +77,9 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
 			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L);
-			
+
 			config.setString(AkkaOptions.WATCH_HEARTBEAT_INTERVAL, "500 ms");
 			config.setString(AkkaOptions.WATCH_HEARTBEAT_PAUSE, "20 s");
 			config.setInteger(AkkaOptions.WATCH_THRESHOLD, 20);
@@ -89,9 +91,9 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 			// for the result
 			List<Long> resultCollection = new ArrayList<Long>();
 
-			final ExecutionEnvironment env = new TestEnvironment(cluster, PARALLELISM, false);
+			final ExecutionEnvironment env = new TestEnvironment(cluster, parallelism, false);
 
-			env.setParallelism(PARALLELISM);
+			env.setParallelism(parallelism);
 			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
 			env.getConfig().disableSysoutLogging();
 
@@ -105,7 +107,6 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 					})
 					.output(new LocalCollectionOutputFormat<Long>(resultCollection));
 
-
 			// simple reference (atomic does not matter) to pass back an exception from the trigger thread
 			final AtomicReference<Throwable> ref = new AtomicReference<Throwable>();
 
@@ -127,7 +128,7 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 
 			// block until all the mappers are actually deployed
 			// the mappers in turn are waiting
-			for (int i = 0; i < PARALLELISM; i++) {
+			for (int i = 0; i < parallelism; i++) {
 				FailingMapper.TASK_TO_COORD_QUEUE.take();
 			}
 
@@ -152,12 +153,12 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 			}
 
 			// wait for the next set of mappers (the recovery ones) to come online
-			for (int i = 0; i < PARALLELISM; i++) {
+			for (int i = 0; i < parallelism; i++) {
 				FailingMapper.TASK_TO_COORD_QUEUE.take();
 			}
 
 			// tell the mappers that they may continue this time
-			for (int i = 0; i < PARALLELISM; i++) {
+			for (int i = 0; i < parallelism; i++) {
 				FailingMapper.COORD_TO_TASK_QUEUE.add(new Object());
 			}
 
@@ -190,7 +191,6 @@ public class TaskManagerFailureRecoveryITCase extends TestLogger {
 
 		private static final BlockingQueue<Object> COORD_TO_TASK_QUEUE = new LinkedBlockingQueue<Object>();
 
-
 		@Override
 		public void open(Configuration parameters) throws Exception {
 			TASK_TO_COORD_QUEUE.add(new Object());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
index 2281572..f700fb8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
@@ -70,8 +71,8 @@ public class TaskManagerProcessFailureBatchRecoveryITCase extends AbstractTaskMa
 		env.getConfig().setExecutionMode(executionMode);
 		env.getConfig().disableSysoutLogging();
 
-		final long NUM_ELEMENTS = 100000L;
-		final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)
+		final long numElements = 100000L;
+		final DataSet<Long> result = env.generateSequence(1, numElements)
 
 				// make sure every mapper is involved (no one is skipped because of lazy split assignment)
 				.rebalance()
@@ -111,6 +112,6 @@ public class TaskManagerProcessFailureBatchRecoveryITCase extends AbstractTaskMa
 				});
 
 		long sum = result.collect().get(0);
-		assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, sum);
+		assertEquals(numElements * (numElements + 1L) / 2L, sum);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
index 987a586..26567a5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java
@@ -18,35 +18,35 @@
 
 package org.apache.flink.test.recovery;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 
+import org.apache.commons.io.FileUtils;
 import org.junit.Assert;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
 import static org.junit.Assert.assertTrue;
 
 /**
  * Test for streaming program behaviour in case of TaskManager failure
  * based on {@link AbstractTaskManagerProcessFailureRecoveryTest}.
  *
- * The logic in this test is as follows:
+ * <p>The logic in this test is as follows:
  *  - The source slowly emits records (every 10 msecs) until the test driver
  *    gives the "go" for regular execution
  *  - The "go" is given after the first taskmanager has been killed, so it can only
@@ -74,7 +74,7 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		env.getConfig().disableSysoutLogging();
 		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
 		env.enableCheckpointing(200);
-		
+
 		env.setStateBackend(new FsStateBackend(tempCheckpointDir.getAbsoluteFile().toURI()));
 
 		DataStream<Long> result = env.addSource(new SleepyDurableGenerateSequence(coordinateDir, DATA_COUNT))
@@ -107,7 +107,7 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		}
 	}
 
-	public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long> 
+	private static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long>
 			implements ListCheckpointed<Long> {
 
 		private static final long SLEEP_TIME = 50;
@@ -175,7 +175,7 @@ public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTa
 		}
 	}
 
-	public static class Mapper extends RichMapFunction<Long, Long> {
+	private static class Mapper extends RichMapFunction<Long, Long> {
 		private boolean markerCreated = false;
 		private File coordinateDir;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
index 1cdb736..c8258a5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/ConsumePipelinedAndBlockingResultITCase.java
@@ -26,6 +26,9 @@ import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+/**
+ * Test join with a slow source.
+ */
 public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBase {
 
 	@Override
@@ -33,7 +36,7 @@ public class ConsumePipelinedAndBlockingResultITCase extends JavaProgramTestBase
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(1);
 
-		DataSet<Tuple1<Long>> pipelinedSource = env.fromElements(new Tuple1<Long>(1l));
+		DataSet<Tuple1<Long>> pipelinedSource = env.fromElements(new Tuple1<Long>(1L));
 
 		DataSet<Tuple1<Long>> slowBlockingSource = env.generateSequence(0, 10).map(
 				new MapFunction<Long, Tuple1<Long>>() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
index 7fb4c82..09be1cd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.runtime;
 
-import akka.actor.ActorSystem;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -30,16 +29,14 @@ import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.testdata.WordCountData;
-
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.NetUtils;
 import org.apache.flink.util.TestLogger;
 
+import akka.actor.ActorSystem;
 import org.junit.Test;
 
-import scala.Some;
-
 import java.io.IOException;
 import java.net.Inet6Address;
 import java.net.InetAddress;
@@ -49,11 +46,16 @@ import java.net.ServerSocket;
 import java.util.Enumeration;
 import java.util.List;
 
-import static org.junit.Assert.*;
+import scala.Some;
+
+import static org.junit.Assert.fail;
 
+/**
+ * Test proper handling of IPv6 address literals in URLs.
+ */
 @SuppressWarnings("serial")
 public class IPv6HostnamesITCase extends TestLogger {
-	
+
 	@Test
 	public void testClusterWithIPv6host() {
 
@@ -63,31 +65,29 @@ public class IPv6HostnamesITCase extends TestLogger {
 			return;
 		}
 
-		
-		
 		LocalFlinkMiniCluster flink = null;
 		try {
 			final String addressString = ipv6address.getHostAddress();
 			log.info("Test will use IPv6 address " + addressString + " for connection tests");
-			
+
 			Configuration conf = new Configuration();
 			conf.setString(JobManagerOptions.ADDRESS, addressString);
 			conf.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, addressString);
 			conf.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			conf.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 			conf.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L);
-			
+
 			flink = new LocalFlinkMiniCluster(conf, false);
 			flink.start();
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(addressString, flink.getLeaderRPCPort());
 			env.setParallelism(4);
 			env.getConfig().disableSysoutLogging();
-			
+
 			// get input data
 			DataSet<String> text = env.fromElements(WordCountData.TEXT.split("\n"));
 
-			DataSet<Tuple2<String, Integer>> counts =text
+			DataSet<Tuple2<String, Integer>> counts = text
 					.flatMap(new FlatMapFunction<String, Tuple2<String, Integer>>() {
 						@Override
 						public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception {
@@ -114,8 +114,7 @@ public class IPv6HostnamesITCase extends TestLogger {
 			}
 		}
 	}
-	
-	
+
 	private Inet6Address getLocalIPv6Address() {
 		try {
 			Enumeration<NetworkInterface> e = NetworkInterface.getNetworkInterfaces();
@@ -126,15 +125,14 @@ public class IPv6HostnamesITCase extends TestLogger {
 				Enumeration<InetAddress> ee = netInterface.getInetAddresses();
 				while (ee.hasMoreElements()) {
 					InetAddress addr = ee.nextElement();
-					
-					
+
 					if (addr instanceof Inet6Address && (!addr.isLoopbackAddress()) && (!addr.isAnyLocalAddress())) {
 						// see if it is possible to bind to the address
 						InetSocketAddress socketAddress = new InetSocketAddress(addr, 0);
-						
+
 						try {
 							log.info("Considering address " + addr);
-							
+
 							// test whether we can bind a socket to that address
 							log.info("Testing whether sockets can bind to " + addr);
 							ServerSocket sock = new ServerSocket();
@@ -158,7 +156,7 @@ public class IPv6HostnamesITCase extends TestLogger {
 					}
 				}
 			}
-			
+
 			return null;
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
index 2c4dca3..82864be 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/JoinDeadlockITCase.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.test.util.JavaProgramTestBase;
+
 import org.junit.Rule;
 import org.junit.rules.Timeout;
 
@@ -60,7 +61,7 @@ public class JoinDeadlockITCase extends JavaProgramTestBase {
 		env.execute();
 	}
 
-	public static class TupleWrapper implements MapFunction<Long, Tuple1<Long>> {
+	private static class TupleWrapper implements MapFunction<Long, Tuple1<Long>> {
 
 		@Override
 		public Tuple1<Long> map(Long l) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
index b5a7b8b..ba3499f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
@@ -32,10 +32,9 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.test.util.JavaProgramTestBase;
-
 import org.apache.flink.util.TestLogger;
-import org.junit.Ignore;
 
+import org.junit.Ignore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,6 +42,9 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * Manually test the throughput of the network stack.
+ */
 @Ignore
 public class NetworkStackThroughputITCase extends TestLogger {
 
@@ -144,7 +146,6 @@ public class NetworkStackThroughputITCase extends TestLogger {
 			return jobGraph;
 		}
 
-
 		@Override
 		protected void testProgram() throws Exception {
 			JobExecutionResult jer = executor.submitJobAndWait(getJobGraph(), false);
@@ -162,7 +163,7 @@ public class NetworkStackThroughputITCase extends TestLogger {
 
 	// ------------------------------------------------------------------------
 
-	public static class SpeedTestProducer extends AbstractInvokable {
+	private static class SpeedTestProducer extends AbstractInvokable {
 
 		@Override
 		public void invoke() throws Exception {
@@ -197,7 +198,7 @@ public class NetworkStackThroughputITCase extends TestLogger {
 		}
 	}
 
-	public static class SpeedTestForwarder extends AbstractInvokable {
+	private static class SpeedTestForwarder extends AbstractInvokable {
 
 		@Override
 		public void invoke() throws Exception {
@@ -221,7 +222,7 @@ public class NetworkStackThroughputITCase extends TestLogger {
 		}
 	}
 
-	public static class SpeedTestConsumer extends AbstractInvokable {
+	private static class SpeedTestConsumer extends AbstractInvokable {
 
 		@Override
 		public void invoke() throws Exception {
@@ -246,7 +247,7 @@ public class NetworkStackThroughputITCase extends TestLogger {
 		}
 	}
 
-	public static class SpeedTestRecord implements IOReadableWritable {
+	private static class SpeedTestRecord implements IOReadableWritable {
 
 		private static final int RECORD_SIZE = 128;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/RegisterTypeWithKryoSerializerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/RegisterTypeWithKryoSerializerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/RegisterTypeWithKryoSerializerITCase.java
index 47f1fc0..ac30402 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/RegisterTypeWithKryoSerializerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/RegisterTypeWithKryoSerializerITCase.java
@@ -18,14 +18,15 @@
 
 package org.apache.flink.test.runtime;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.Serializer;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -35,6 +36,9 @@ import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 
+/**
+ * Test registering types with Kryo.
+ */
 @RunWith(Parameterized.class)
 public class RegisterTypeWithKryoSerializerITCase extends MultipleProgramsTestBase {
 
@@ -73,7 +77,7 @@ public class RegisterTypeWithKryoSerializerITCase extends MultipleProgramsTestBa
 		compareResultCollections(expected, mapped.collect(), new Comparator<TestClass>() {
 			@Override
 			public int compare(TestClass o1, TestClass o2) {
-				return (int)(o1.getValue() - o2.getValue());
+				return (int) (o1.getValue() - o2.getValue());
 			}
 		});
 	}
@@ -108,7 +112,7 @@ public class RegisterTypeWithKryoSerializerITCase extends MultipleProgramsTestBa
 
 		@Override
 		public int hashCode() {
-			return (int)value;
+			return (int) value;
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
index 7729138..030669f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/SelfJoinDeadlockITCase.java
@@ -27,6 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Rule;
 import org.junit.rules.Timeout;
 
@@ -62,7 +63,7 @@ public class SelfJoinDeadlockITCase extends JavaProgramTestBase {
 	}
 
 	@SuppressWarnings("serial")
-	public static class Joiner implements FlatJoinFunction<Tuple3<Integer, Integer, String>, Tuple3<Integer, Integer, String>, Tuple5<Integer, Integer, Integer, String, String>> {
+	private static class Joiner implements FlatJoinFunction<Tuple3<Integer, Integer, String>, Tuple3<Integer, Integer, String>, Tuple5<Integer, Integer, Integer, String, String>> {
 
 		@Override
 		public void join(Tuple3<Integer, Integer, String> in1, Tuple3<Integer, Integer, String> in2, Collector<Tuple5<Integer, Integer, Integer, String, String>> out) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
index 37e89e9..1fbbdb2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.test.runtime.leaderelection;
 
-import akka.actor.ActorSystem;
-import akka.actor.Kill;
-import akka.actor.PoisonPill;
-import org.apache.curator.test.TestingServer;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.AkkaOptions;
 import org.apache.flink.configuration.ConfigConstants;
@@ -43,6 +39,11 @@ import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAl
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.actor.Kill;
+import akka.actor.PoisonPill;
+import org.apache.curator.test.TestingServer;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -50,17 +51,21 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 import scala.concurrent.impl.Promise;
 
-import java.io.File;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+/**
+ * Test the election of a new JobManager leader.
+ */
 public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 	private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION();
@@ -250,7 +255,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		}
 	}
 
-	public static class JobSubmitterRunnable implements Runnable {
+	private static class JobSubmitterRunnable implements Runnable {
 		private static final Logger LOG = LoggerFactory.getLogger(JobSubmitterRunnable.class);
 		boolean finished = false;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
index 6399e2e..40c4e84 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/minicluster/LocalFlinkMiniClusterITCase.java
@@ -18,39 +18,42 @@
 
 package org.apache.flink.test.runtime.minicluster;
 
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
 import org.junit.Test;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.forkjoin.ForkJoinPool;
-import scala.concurrent.impl.ExecutionContextImpl;
 
 import java.lang.reflect.Field;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.forkjoin.ForkJoinPool;
+import scala.concurrent.impl.ExecutionContextImpl;
+
 import static org.junit.Assert.fail;
 
+/**
+ * Integration tests for {@link LocalFlinkMiniCluster}.
+ */
 public class LocalFlinkMiniClusterITCase extends TestLogger {
 
-	private static String[] ALLOWED_THREAD_PREFIXES = { };
+	private static final String[] ALLOWED_THREAD_PREFIXES = { };
 
 	@Test
 	public void testLocalFlinkMiniClusterWithMultipleTaskManagers() {
-		
+
 		final ActorSystem system = ActorSystem.create("Testkit", AkkaUtils.getDefaultAkkaConfig());
 		LocalFlinkMiniCluster miniCluster = null;
 
@@ -64,8 +67,7 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 			Thread.enumerate(allThreads);
 			threadsBefore.addAll(Arrays.asList(allThreads));
 		}
-		
-		
+
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
@@ -93,12 +95,11 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 								JobManagerMessages.getRequestTotalNumberOfSlots(),
 								selfGateway);
 
-						expectMsgEquals(TestingUtils.TESTING_DURATION(), numTMs*numSlots);
+						expectMsgEquals(TestingUtils.TESTING_DURATION(), numTMs * numSlots);
 					}
 				};
 			}};
 
-
 		} finally {
 			if (miniCluster != null) {
 				miniCluster.stop();
@@ -113,7 +114,7 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 		try {
 			Field f = ExecutionContextImpl.class.getDeclaredField("executor");
 			f.setAccessible(true);
-			
+
 			Object exec = ExecutionContext$.MODULE$.global();
 			ForkJoinPool executor = (ForkJoinPool) f.get(exec);
 			executor.shutdownNow();
@@ -122,14 +123,14 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 			System.err.println("Cannot test proper thread shutdown for local execution.");
 			return;
 		}
-		
+
 		// check for remaining threads
 		// we need to check repeatedly for a while, because some threads shut down slowly
-		
+
 		long deadline = System.currentTimeMillis() + 30000;
 		boolean foundThreads = true;
 		String threadName = "";
-		
+
 		while (System.currentTimeMillis() < deadline) {
 			// check that no additional threads remain
 			final Thread[] threadsAfter = new Thread[Thread.activeCount()];
@@ -146,7 +147,7 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 							break;
 						}
 					}
-					
+
 					if (!allowed) {
 						foundThreads = true;
 						threadName = t.toString();
@@ -154,7 +155,7 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 					}
 				}
 			}
-			
+
 			if (foundThreads) {
 				try {
 					Thread.sleep(500);
@@ -163,7 +164,7 @@ public class LocalFlinkMiniClusterITCase extends TestLogger {
 				break;
 			}
 		}
-		
+
 		if (foundThreads) {
 			fail("Thread " + threadName + " was started by the mini cluster, but not shut down");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java
index cbb56d0..eddc0a8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state;
 
 import org.apache.flink.api.common.functions.FilterFunction;
@@ -28,6 +29,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
@@ -129,7 +131,7 @@ public class ManualWindowSpeedITCase extends StreamingMultipleProgramsTestBase {
 
 		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
 		env.setParallelism(1);
-		
+
 		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
 
 		env.addSource(new InfiniteTupleSource(10_000))

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/StateHandleSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/StateHandleSerializationTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/StateHandleSerializationTest.java
index 77a9b2e..eb964ec 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/StateHandleSerializationTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/StateHandleSerializationTest.java
@@ -19,22 +19,23 @@
 package org.apache.flink.test.state;
 
 import org.apache.flink.runtime.state.StateObject;
-import org.junit.Test;
 
+import org.junit.Test;
 import org.reflections.Reflections;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.util.Set;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
 
+/**
+ * This test validates that all subclasses of {@link StateObject} have a proper
+ * serial version UID.
+ */
 public class StateHandleSerializationTest {
 
-	/**
-	 * This test validates that all subclasses of {@link StateObject} have a proper
-	 * serial version UID.
-	 */
 	@Test
 	public void ensureStateHandlesHaveSerialVersionUID() {
 		try {
@@ -55,7 +56,7 @@ public class StateHandleSerializationTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	private static void validataSerialVersionUID(Class<?> clazz) {
 		// all non-interface types must have a serial version UID
 		if (!clazz.isInterface()) {
@@ -65,11 +66,10 @@ public class StateHandleSerializationTest {
 			try {
 				Field versionUidField = clazz.getDeclaredField("serialVersionUID");
 
-				// check conditions first via "if" to prevent always constructing expensive error messages 
+				// check conditions first via "if" to prevent always constructing expensive error messages
 				if (!(Modifier.isPrivate(versionUidField.getModifiers()) &&
 						Modifier.isStatic(versionUidField.getModifiers()) &&
-						Modifier.isFinal(versionUidField.getModifiers())))
-				{
+						Modifier.isFinal(versionUidField.getModifiers()))) {
 					fail(clazz.getName() + " - serialVersionUID is not 'private static final'");
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
index b6dfb02..3d78242 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
@@ -15,11 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.PoisonPill;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -48,27 +46,32 @@ import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.net.URL;
+import java.util.concurrent.TimeUnit;
+
 import scala.Option;
 import scala.Tuple2;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
-import java.io.File;
-import java.net.URL;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Abstract class to verify that it is possible to migrate a savepoint across upgraded Flink versions and that the
  * topology can be modified from that point on.
- * 
- * The verification is done in 2 Steps:
+ *
+ * <p>The verification is done in 2 Steps:
  * Step 1: Migrate the job to the newer version by submitting the same job used for the old version savepoint, and create a new savepoint.
  * Step 2: Modify the job topology, and restore from the savepoint created in step 1.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/ExecutionMode.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/ExecutionMode.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/ExecutionMode.java
index ae9fb21..1705de1 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/ExecutionMode.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/ExecutionMode.java
@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore;
 
 /**
  * Enum to control function behavior for the different test stages.
- * 
+ *
  * {@link ExecutionMode#GENERATE} should be used when creating the savepoint.
  * {@link ExecutionMode#MIGRATE} should be used when migrating the savepoint to a newer version.
  * {@link ExecutionMode#RESTORE} should be used when restoring from the migrated newer version savepoint.

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedComplexChainTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedComplexChainTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedComplexChainTest.java
index 605722d..45dbc78 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedComplexChainTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedComplexChainTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.keyed;
 
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -22,6 +23,9 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.state.operator.restore.ExecutionMode;
 
+/**
+ * Test state restoration for a keyed operator restore tests.
+ */
 public class KeyedComplexChainTest extends AbstractKeyedOperatorRestoreTestBase {
 
 	public KeyedComplexChainTest(String savepointPath) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
index 3b23c01..076feda 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.keyed;
 
 import org.apache.flink.api.common.functions.RichMapFunction;
@@ -37,6 +38,7 @@ import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import org.apache.flink.test.state.operator.restore.ExecutionMode;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 
 import java.util.Arrays;
@@ -47,7 +49,7 @@ import java.util.List;
  * Savepoint generator to create the savepoint used by the {@link AbstractKeyedOperatorRestoreTestBase}.
  * Switch to specific version branches and run this job to create savepoints of different Flink versions.
  *
- * The job should be cancelled manually through the REST API using the cancel-with-savepoint operation.
+ * <p>The job should be cancelled manually through the REST API using the cancel-with-savepoint operation.
  */
 public class KeyedJob {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java
index 22fa7b2..f07bd4d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/AbstractNonKeyedOperatorRestoreTestBase.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainBreakTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainBreakTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainBreakTest.java
index 8055833..4e074ea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainBreakTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainBreakTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthDecreaseTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthDecreaseTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthDecreaseTest.java
index 3235387..af81c0e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthDecreaseTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthDecreaseTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthIncreaseTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthIncreaseTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthIncreaseTest.java
index a10f99c..1f07e0b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthIncreaseTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainLengthIncreaseTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainOrderTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainOrderTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainOrderTest.java
index 0baa233..61e366d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainOrderTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainOrderTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainUnionTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainUnionTest.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainUnionTest.java
index 0d21e8a..456e746 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainUnionTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/ChainUnionTest.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.streaming.api.datastream.DataStream;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
index 1c55681..1b0ed45 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/unkeyed/NonKeyedJob.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.state.operator.restore.unkeyed;
 
 import org.apache.flink.api.common.functions.MapFunction;
@@ -31,6 +32,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.test.state.operator.restore.ExecutionMode;
+
 import org.junit.Assert;
 
 import java.util.Arrays;
@@ -40,7 +42,7 @@ import java.util.List;
  * Savepoint generator to create the savepoint used by the {@link AbstractNonKeyedOperatorRestoreTestBase}.
  * Switch to specific version branches and run this job to create savepoints of different Flink versions.
  *
- * The job should be cancelled manually through the REST API using the cancel-with-savepoint operation.
+ * <p>The job should be cancelled manually through the REST API using the cancel-with-savepoint operation.
  */
 public class NonKeyedJob {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
index 8ea1bd8..6d2f8c5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java
@@ -30,28 +30,36 @@ import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.async.AsyncFunction;
 import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
+import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.async.collector.AsyncCollector;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.MathUtils;
-import org.junit.*;
 
-import java.util.*;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * Integration tests for streaming operators.
+ */
 public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase {
 
-
 	/**
 	 * Tests the proper functioning of the streaming fold operator. For this purpose, a stream
-	 * of Tuple2<Integer, Integer> is created. The stream is grouped according to the first tuple
-	 * value. Each group is folded where the second tuple value is summed up.
+	 * of {@code Tuple2<Integer, Integer>} is created. The stream is grouped according to the
+	 * first tuple value. Each group is folded where the second tuple value is summed.
 	 *
-	 * This test relies on the hash function used by the {@link DataStream#keyBy}, which is
+	 * <p>This test relies on the hash function used by the {@link DataStream#keyBy}, which is
 	 * assumed to be {@link MathUtils#murmurHash}.
 	 */
 	@Test
@@ -99,7 +107,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		final List<Integer> actualResult1 = new ArrayList<>();
 		MemorySinkFunction.registerCollection(0, actualResult1);
 
-		splittedResult.select("0").map(new MapFunction<Tuple2<Integer,Integer>, Integer>() {
+		splittedResult.select("0").map(new MapFunction<Tuple2<Integer, Integer>, Integer>() {
 			private static final long serialVersionUID = 2114608668010092995L;
 
 			@Override
@@ -188,7 +196,7 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 		Collection<Integer> expected = new ArrayList<>(10);
 
 		for (int i = 0; i < numElements; i++) {
-			expected.add(42 + i );
+			expected.add(42 + i);
 		}
 
 		env.execute();
@@ -270,11 +278,10 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		unorderedResult.addSink(sinkFunction2);
 
-
 		Collection<Integer> expected = new ArrayList<>(10);
 
 		for (int i = 0; i < numElements; i++) {
-			expected.add(i+i);
+			expected.add(i + i);
 		}
 
 		env.execute();
@@ -306,7 +313,6 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 			this.numElements = numElements;
 		}
 
-
 		@Override
 		public void run(SourceContext<Tuple2<Integer, NonSerializable>> ctx) throws Exception {
 			for (int i = 0; i < numElements; i++) {
@@ -340,7 +346,6 @@ public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase
 
 		@Override
 
-
 		public void cancel() {
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java
index 61dddb8..a95865a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/CsvOutputFormatITCase.java
@@ -24,6 +24,9 @@ import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.testfunctions.Tokenizer;
 
+/**
+ * Integration tests for {@link org.apache.flink.api.java.io.CsvOutputFormat}.
+ */
 public class CsvOutputFormatITCase extends StreamingProgramTestBase {
 
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java
index 6ea864e..7f0ebc9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/outputformat/TextOutputFormatITCase.java
@@ -24,6 +24,9 @@ import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.testfunctions.Tokenizer;
 
+/**
+ * Integration tests for {@link org.apache.flink.api.java.io.TextOutputFormat}.
+ */
 public class TextOutputFormatITCase extends StreamingProgramTestBase {
 
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java
index d21985b..92ec748 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java
@@ -29,6 +29,9 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertNotEquals;
 
+/**
+ * Test creation of context for chained streaming operators.
+ */
 @SuppressWarnings("serial")
 public class ChainedRuntimeContextITCase extends StreamingMultipleProgramsTestBase {
 	private static RuntimeContext srcContext;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoGroupJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoGroupJoinITCase.java
index a82b965..74f84c4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoGroupJoinITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoGroupJoinITCase.java
@@ -39,6 +39,7 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -48,6 +49,9 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * Integration tests for windowed join / coGroup operators.
+ */
 @SuppressWarnings("serial")
 public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 
@@ -108,12 +112,11 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 			}
 		}).assignTimestampsAndWatermarks(new Tuple2TimestampExtractor());
 
-
 		source1.coGroup(source2)
 				.where(new Tuple2KeyExtractor())
 				.equalTo(new Tuple2KeyExtractor())
 				.window(TumblingEventTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+				.apply(new CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String>() {
 					@Override
 					public void coGroup(Iterable<Tuple2<String, Integer>> first,
 							Iterable<Tuple2<String, Integer>> second,
@@ -180,7 +183,7 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 
 			@Override
 			public void cancel() {}
-			
+
 		}).assignTimestampsAndWatermarks(new Tuple3TimestampExtractor());
 
 		DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
@@ -201,9 +204,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 
 			@Override
 			public void cancel() {}
-			
-		}).assignTimestampsAndWatermarks(new Tuple3TimestampExtractor());
 
+		}).assignTimestampsAndWatermarks(new Tuple3TimestampExtractor());
 
 		source1.join(source2)
 				.where(new Tuple3KeyExtractor())
@@ -351,7 +353,7 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 			.where(new Tuple2KeyExtractor())
 			.equalTo(new Tuple2KeyExtractor())
 			.window(TumblingEventTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-			.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+			.apply(new CoGroupFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String>() {
 				@Override
 				public void coGroup(Iterable<Tuple2<String, Integer>> first,
 									Iterable<Tuple2<String, Integer>> second,
@@ -372,7 +374,7 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 	}
 
 	private static class Tuple2TimestampExtractor implements AssignerWithPunctuatedWatermarks<Tuple2<String, Integer>> {
-		
+
 		@Override
 		public long extractTimestamp(Tuple2<String, Integer> element, long previousTimestamp) {
 			return element.f1;
@@ -397,7 +399,7 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	private static class Tuple2KeyExtractor implements KeySelector<Tuple2<String,Integer>, String> {
+	private static class Tuple2KeyExtractor implements KeySelector<Tuple2<String, Integer>, String> {
 
 		@Override
 		public String getKey(Tuple2<String, Integer> value) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoStreamITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoStreamITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoStreamITCase.java
index 360ceb3..178363d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoStreamITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/CoStreamITCase.java
@@ -36,6 +36,9 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Integration tests for co-streams.
+ */
 @SuppressWarnings("serial")
 public class CoStreamITCase extends StreamingMultipleProgramsTestBase {
 
@@ -56,7 +59,7 @@ public class CoStreamITCase extends StreamingMultipleProgramsTestBase {
 						return true;
 					}
 				})
-				
+
 				.keyBy(new KeySelector<Integer, Integer>() {
 					@Override
 					public Integer getKey(Integer value) throws Exception {
@@ -96,7 +99,7 @@ public class CoStreamITCase extends StreamingMultipleProgramsTestBase {
 					public void flatMap1(Integer value, Collector<String> out) throws Exception {
 						out.collect(value.toString());
 					}
-		
+
 					@Override
 					public void flatMap2(Tuple2<Integer, Integer> value, Collector<String> out) throws Exception {
 						out.collect(value.toString());
@@ -105,7 +108,6 @@ public class CoStreamITCase extends StreamingMultipleProgramsTestBase {
 
 		connected.addSink(resultSink);
 
-		
 		env.execute();
 
 		List<String> expected = Arrays.asList("(1,2)", "(3,4)", "(5,6)", "1", "3", "5");

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DataStreamPojoITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DataStreamPojoITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DataStreamPojoITCase.java
index 0949c68..8a18ff3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DataStreamPojoITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DataStreamPojoITCase.java
@@ -23,29 +23,30 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
 
 /**
- * Integration test for streaming programs using POJOs and key selectors
+ * Integration test for streaming programs using POJOs and key selectors.
  *
- * See FLINK-3697
+ * <p>See FLINK-3697
  */
 public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 	static List<Data> elements = new ArrayList<>();
 	static {
-		elements.add(new Data(0,0,0));
-		elements.add(new Data(0,0,0));
-		elements.add(new Data(1,1,1));
-		elements.add(new Data(1,1,1));
-		elements.add(new Data(2,2,3));
-		elements.add(new Data(2,2,3));
+		elements.add(new Data(0, 0, 0));
+		elements.add(new Data(0, 0, 0));
+		elements.add(new Data(1, 1, 1));
+		elements.add(new Data(1, 1, 1));
+		elements.add(new Data(2, 2, 3));
+		elements.add(new Data(2, 2, 3));
 	}
 
 	/**
-	 * Test composite key on the Data POJO (with nested fields)
+	 * Test composite key on the Data POJO (with nested fields).
 	 */
 	@Test
 	public void testCompositeKeyOnNestedPojo() throws Exception {
@@ -64,22 +65,22 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 					Data[] first = new Data[3];
 					@Override
 					public void flatMap(Data value, Collector<Data> out) throws Exception {
-						if(first[value.aaa] == null) {
+						if (first[value.aaa] == null) {
 							first[value.aaa] = value;
-							if(value.sum != 1) {
+							if (value.sum != 1) {
 								throw new RuntimeException("Expected the sum to be one");
 							}
 						} else {
-							if(value.sum != 2) {
+							if (value.sum != 2) {
 								throw new RuntimeException("Expected the sum to be two");
 							}
-							if(first[value.aaa].aaa != value.aaa) {
+							if (first[value.aaa].aaa != value.aaa) {
 								throw new RuntimeException("aaa key wrong");
 							}
-							if(first[value.aaa].abc != value.abc) {
+							if (first[value.aaa].abc != value.abc) {
 								throw new RuntimeException("abc key wrong");
 							}
-							if(first[value.aaa].wxyz != value.wxyz) {
+							if (first[value.aaa].wxyz != value.wxyz) {
 								throw new RuntimeException("wxyz key wrong");
 							}
 						}
@@ -92,7 +93,7 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 	}
 
 	/**
-	 * Test composite & nested key on the Data POJO
+	 * Test composite & nested key on the Data POJO.
 	 */
 	@Test
 	public void testNestedKeyOnNestedPojo() throws Exception {
@@ -111,25 +112,25 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 					Data[] first = new Data[3];
 					@Override
 					public void flatMap(Data value, Collector<Data> out) throws Exception {
-						if(value.stats.count != 123) {
+						if (value.stats.count != 123) {
 							throw new RuntimeException("Wrong value for value.stats.count");
 						}
-						if(first[value.aaa] == null) {
+						if (first[value.aaa] == null) {
 							first[value.aaa] = value;
-							if(value.sum != 1) {
+							if (value.sum != 1) {
 								throw new RuntimeException("Expected the sum to be one");
 							}
 						} else {
-							if(value.sum != 2) {
+							if (value.sum != 2) {
 								throw new RuntimeException("Expected the sum to be two");
 							}
-							if(first[value.aaa].aaa != value.aaa) {
+							if (first[value.aaa].aaa != value.aaa) {
 								throw new RuntimeException("aaa key wrong");
 							}
-							if(first[value.aaa].abc != value.abc) {
+							if (first[value.aaa].abc != value.abc) {
 								throw new RuntimeException("abc key wrong");
 							}
-							if(first[value.aaa].wxyz != value.wxyz) {
+							if (first[value.aaa].wxyz != value.wxyz) {
 								throw new RuntimeException("wxyz key wrong");
 							}
 						}
@@ -157,13 +158,13 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 				Data[] first = new Data[3];
 				@Override
 				public void flatMap(Data value, Collector<Data> out) throws Exception {
-					if(first[value.aaa] == null) {
+					if (first[value.aaa] == null) {
 						first[value.aaa] = value;
-						if(value.stats.count != 123) {
+						if (value.stats.count != 123) {
 							throw new RuntimeException("Expected stats.count to be 123");
 						}
 					} else {
-						if(value.stats.count != 2 * 123) {
+						if (value.stats.count != 2 * 123) {
 							throw new RuntimeException("Expected stats.count to be 2 * 123");
 						}
 					}
@@ -183,7 +184,9 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 		dataStream.keyBy("aaa", "stats.count").sum("stats.nonExistingField");
 	}
 
-
+	/**
+	 * POJO.
+	 */
 	public static class Data {
 		public int sum; // sum
 		public int aaa; // keyBy
@@ -196,6 +199,7 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 
 		public Data() {
 		}
+
 		public Data(int aaa, int abc, int wxyz) {
 			this.sum = 1;
 			this.aaa = aaa;
@@ -215,6 +219,10 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 					'}';
 		}
 	}
+
+	/**
+	 * POJO.
+	 */
 	public static class Policy {
 		public short a;
 		public short b;
@@ -224,6 +232,9 @@ public class DataStreamPojoITCase extends StreamingMultipleProgramsTestBase {
 		public Policy() {}
 	}
 
+	/**
+	 * POJO.
+	 */
 	public static class Stats {
 		public long count;
 		public float a;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DirectedOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DirectedOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DirectedOutputITCase.java
index 8b84112..15901da 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DirectedOutputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/DirectedOutputITCase.java
@@ -31,6 +31,9 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Integration tests for a streaming {@link OutputSelector}.
+ */
 public class DirectedOutputITCase extends StreamingMultipleProgramsTestBase {
 
 	private static final String TEN = "ten";

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/IterateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/IterateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/IterateITCase.java
index e49f832..0f6b2bc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/IterateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/IterateITCase.java
@@ -50,7 +50,6 @@ import org.apache.flink.util.Collector;
 import org.apache.flink.util.MathUtils;
 
 import org.junit.Test;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,12 +62,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Tests for streaming iterations.
+ */
 @SuppressWarnings({ "unchecked", "unused", "serial" })
 public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 	private static final Logger LOG = LoggerFactory.getLogger(IterateITCase.class);
 
-	private static boolean iterated[];
+	private static boolean[] iterated;
 
 	@Test(expected = UnsupportedOperationException.class)
 	public void testIncorrectParallelism() throws Exception {
@@ -78,7 +80,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		DataStream<Integer> source = env.fromElements(1, 10);
 
 		IterativeStream<Integer> iter1 = source.iterate();
-		SingleOutputStreamOperator<Integer> map1 = iter1.map(NoOpIntMap);
+		SingleOutputStreamOperator<Integer> map1 = iter1.map(noOpIntMap);
 		iter1.closeWith(map1).print();
 	}
 
@@ -88,15 +90,14 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source.iterate();
 
-		iter1.closeWith(iter1.map(NoOpIntMap));
-		iter1.closeWith(iter1.map(NoOpIntMap));
+		iter1.closeWith(iter1.map(noOpIntMap));
+		iter1.closeWith(iter1.map(noOpIntMap));
 	}
 
-
 	@Test(expected = UnsupportedOperationException.class)
 	public void testDifferingParallelism() throws Exception {
 
@@ -104,29 +105,26 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 		// introduce dummy mapper to get to correct parallelism
 		DataStream<Integer> source = env.fromElements(1, 10)
-				.map(NoOpIntMap);
+				.map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source.iterate();
 
-
-		iter1.closeWith(iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2));
+		iter1.closeWith(iter1.map(noOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2));
 
 	}
 
-
 	@Test(expected = UnsupportedOperationException.class)
 	public void testCoDifferingParallelism() throws Exception {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap);
 
 		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
-
-		coIter.closeWith(coIter.map(NoOpIntCoMap).setParallelism(DEFAULT_PARALLELISM / 2));
+		coIter.closeWith(coIter.map(noOpIntCoMap).setParallelism(DEFAULT_PARALLELISM / 2));
 
 	}
 
@@ -139,13 +137,12 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source.iterate();
 		IterativeStream<Integer> iter2 = source.iterate();
 
-
-		iter2.closeWith(iter1.map(NoOpIntMap));
+		iter2.closeWith(iter1.map(noOpIntMap));
 
 	}
 
@@ -158,14 +155,13 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
 		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source.iterate();
 		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
 				Integer.class);
 
-
-		coIter.closeWith(iter1.map(NoOpIntMap));
+		coIter.closeWith(iter1.map(noOpIntMap));
 
 	}
 
@@ -174,11 +170,11 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source.iterate();
 
-		iter1.map(NoOpIntMap).print();
+		iter1.map(noOpIntMap).print();
 
 		env.execute();
 	}
@@ -187,14 +183,14 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 	public void testImmutabilityWithCoiteration() {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap); // for rebalance
+		DataStream<Integer> source = env.fromElements(1, 10).map(noOpIntMap); // for rebalance
 
 		IterativeStream<Integer> iter1 = source.iterate();
 		// Calling withFeedbackType should create a new iteration
 		ConnectedIterativeStreams<Integer, String> iter2 = iter1.withFeedbackType(String.class);
 
-		iter1.closeWith(iter1.map(NoOpIntMap)).print();
-		iter2.closeWith(iter2.map(NoOpCoMap)).print();
+		iter1.closeWith(iter1.map(noOpIntMap)).print();
+		iter2.closeWith(iter2.map(noOpCoMap)).print();
 
 		StreamGraph graph = env.getStreamGraph();
 
@@ -211,23 +207,23 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
 				.shuffle()
-				.map(NoOpIntMap).name("ParallelizeMapShuffle");
+				.map(noOpIntMap).name("ParallelizeMapShuffle");
 		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap).name("ParallelizeMapRebalance");
+				.map(noOpIntMap).name("ParallelizeMapRebalance");
 
 		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
 
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2);
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap).name("IterForwardMap");
-		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink<Integer>());
-		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
+		DataStream<Integer> head1 = iter1.map(noOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2);
+		DataStream<Integer> head2 = iter1.map(noOpIntMap).name("IterForwardMap");
+		DataStreamSink<Integer> head3 = iter1.map(noOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink<Integer>());
+		DataStreamSink<Integer> head4 = iter1.map(noOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
 
 		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap).name("EvenOddSourceMap")
+				.map(noOpIntMap).name("EvenOddSourceMap")
 				.split(new EvenOddOutputSelector());
 
 		iter1.closeWith(source3.select("even").union(
-				head1.rebalance().map(NoOpIntMap).broadcast(), head2.shuffle()));
+				head1.rebalance().map(noOpIntMap).broadcast(), head2.shuffle()));
 
 		StreamGraph graph = env.getStreamGraph();
 
@@ -291,30 +287,30 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
 				.shuffle()
-				.map(NoOpIntMap);
+				.map(noOpIntMap);
 
 		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap);
+				.map(noOpIntMap);
 
 		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
 
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("map1");
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap)
+		DataStream<Integer> head1 = iter1.map(noOpIntMap).name("map1");
+		DataStream<Integer> head2 = iter1.map(noOpIntMap)
 				.setParallelism(DEFAULT_PARALLELISM / 2)
 				.name("shuffle").rebalance();
-		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2)
+		DataStreamSink<Integer> head3 = iter1.map(noOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2)
 				.addSink(new ReceiveCheckNoOpSink<Integer>());
-		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
+		DataStreamSink<Integer> head4 = iter1.map(noOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
 
 		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap)
+				.map(noOpIntMap)
 				.name("split")
 				.split(new EvenOddOutputSelector());
 
 		iter1.closeWith(
 				source3.select("even").union(
-						head1.map(NoOpIntMap).name("bc").broadcast(),
-						head2.map(NoOpIntMap).shuffle()));
+						head1.map(noOpIntMap).name("bc").broadcast(),
+						head2.map(noOpIntMap).shuffle()));
 
 		StreamGraph graph = env.getStreamGraph();
 
@@ -329,7 +325,6 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		assertEquals(4, itSource.getOutEdges().size());
 		assertEquals(3, itSink.getInEdges().size());
 
-
 		assertEquals(itSource.getParallelism(), itSink.getParallelism());
 
 		for (StreamEdge edge : itSource.getOutEdges()) {
@@ -384,13 +379,13 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 				iterated = new boolean[DEFAULT_PARALLELISM];
 
 				DataStream<Boolean> source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
-						.map(NoOpBoolMap).name("ParallelizeMap");
+						.map(noOpBoolMap).name("ParallelizeMap");
 
 				IterativeStream<Boolean> iteration = source.iterate(3000 * timeoutScale);
 
-				DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).map(NoOpBoolMap);
+				DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).map(noOpBoolMap);
 
-				iteration.map(NoOpBoolMap).addSink(new ReceiveCheckNoOpSink());
+				iteration.map(noOpBoolMap).addSink(new ReceiveCheckNoOpSink());
 
 				iteration.closeWith(increment).addSink(new ReceiveCheckNoOpSink());
 
@@ -426,11 +421,10 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 				env.setParallelism(2);
 
 				DataStream<String> otherSource = env.fromElements("1000", "2000")
-						.map(NoOpStrMap).name("ParallelizeMap");
-
+						.map(noOpStrMap).name("ParallelizeMap");
 
 				ConnectedIterativeStreams<Integer, String> coIt = env.fromElements(0, 0)
-						.map(NoOpIntMap).name("ParallelizeMap")
+						.map(noOpIntMap).name("ParallelizeMap")
 						.iterate(2000 * timeoutScale)
 						.withFeedbackType("String");
 
@@ -510,7 +504,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 	 * This test relies on the hash function used by the {@link DataStream#keyBy}, which is
 	 * assumed to be {@link MathUtils#murmurHash}.
 	 *
-	 * For the test to pass all FlatMappers must see at least two records in the iteration,
+	 * <p>For the test to pass all FlatMappers must see at least two records in the iteration,
 	 * which can only be achieved if the hashed values of the input keys map to a complete
 	 * congruence system. Given that the test is designed for 3 parallel FlatMapper instances
 	 * keys chosen from the [1,3] range are a suitable choice.
@@ -535,7 +529,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 				};
 
 				DataStream<Integer> source = env.fromElements(1, 2, 3)
-						.map(NoOpIntMap).name("ParallelizeMap");
+						.map(noOpIntMap).name("ParallelizeMap");
 
 				IterativeStream<Integer> it = source.keyBy(key).iterate(3000 * timeoutScale);
 
@@ -563,7 +557,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 					}
 				});
 
-				it.closeWith(head.keyBy(key).union(head.map(NoOpIntMap).keyBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
+				it.closeWith(head.keyBy(key).union(head.map(noOpIntMap).keyBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
 
 				env.execute();
 
@@ -593,8 +587,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 				env.enableCheckpointing();
 
 				DataStream<Boolean> source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
-						.map(NoOpBoolMap).name("ParallelizeMap");
-
+						.map(noOpBoolMap).name("ParallelizeMap");
 
 				IterativeStream<Boolean> iteration = source.iterate(3000 * timeoutScale);
 
@@ -637,7 +630,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	public static final class IterationHead extends RichFlatMapFunction<Boolean, Boolean> {
+	private static final class IterationHead extends RichFlatMapFunction<Boolean, Boolean> {
 		public void flatMap(Boolean value, Collector<Boolean> out) throws Exception {
 			int indx = getRuntimeContext().getIndexOfThisSubtask();
 			if (value) {
@@ -648,7 +641,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	public static CoMapFunction<Integer, String, String> NoOpCoMap = new CoMapFunction<Integer, String, String>() {
+	public static CoMapFunction<Integer, String, String> noOpCoMap = new CoMapFunction<Integer, String, String>() {
 
 		public String map1(Integer value) throws Exception {
 			return value.toString();
@@ -659,9 +652,9 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 		}
 	};
 
-	public static MapFunction<Integer, Integer> NoOpIntMap = new NoOpIntMap();
+	public static MapFunction<Integer, Integer> noOpIntMap = new NoOpIntMap();
 
-	public static MapFunction<String, String> NoOpStrMap = new MapFunction<String, String>() {
+	public static MapFunction<String, String> noOpStrMap = new MapFunction<String, String>() {
 
 		public String map(String value) throws Exception {
 			return value;
@@ -669,7 +662,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 	};
 
-	public static CoMapFunction<Integer, Integer, Integer> NoOpIntCoMap = new CoMapFunction<Integer, Integer, Integer>() {
+	public static CoMapFunction<Integer, Integer, Integer> noOpIntCoMap = new CoMapFunction<Integer, Integer, Integer>() {
 
 		public Integer map1(Integer value) throws Exception {
 			return value;
@@ -681,7 +674,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 	};
 
-	public static MapFunction<Boolean, Boolean> NoOpBoolMap = new MapFunction<Boolean, Boolean>() {
+	public static MapFunction<Boolean, Boolean> noOpBoolMap = new MapFunction<Boolean, Boolean>() {
 
 		public Boolean map(Boolean value) throws Exception {
 			return value;
@@ -689,7 +682,7 @@ public class IterateITCase extends StreamingMultipleProgramsTestBase {
 
 	};
 
-	public static class TestSink implements SinkFunction<String> {
+	private static class TestSink implements SinkFunction<String> {
 
 		private static final long serialVersionUID = 1L;
 		public static List<String> collected = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/OutputSplitterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/OutputSplitterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/OutputSplitterITCase.java
index 0902a3c..988e721 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/OutputSplitterITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/OutputSplitterITCase.java
@@ -31,6 +31,9 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Integration tests for a streaming {@link OutputSelector}.
+ */
 public class OutputSplitterITCase extends StreamingMultipleProgramsTestBase {
 
 	private static ArrayList<Integer> expectedSplitterResult = new ArrayList<Integer>();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
index 3a125ba..92d268e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java
@@ -72,7 +72,6 @@ public class PartitionerITCase extends StreamingMultipleProgramsTestBase {
 		env.execute();
 	}
 
-
 	@Test
 	public void partitionerTest() {
 
@@ -89,7 +88,6 @@ public class PartitionerITCase extends StreamingMultipleProgramsTestBase {
 		TestListResultSink<Tuple2<Integer, String>> globalPartitionResultSink =
 				new TestListResultSink<Tuple2<Integer, String>>();
 
-
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(3);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java
index d33a2b1..1a49aef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java
@@ -35,6 +35,9 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 
+/**
+ * Integration tests for connected streams.
+ */
 public class SelfConnectionITCase extends StreamingMultipleProgramsTestBase {
 
 	/**
@@ -136,7 +139,6 @@ public class SelfConnectionITCase extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-
 		stringMap.connect(longMap).map(new CoMapFunction<String, Long, String>() {
 
 			private static final long serialVersionUID = 1L;


[18/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
deleted file mode 100644
index b952bc2..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WebLogAnalysisITCase.java
+++ /dev/null
@@ -1,53 +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.exampleScalaPrograms;
-
-
-import org.apache.flink.examples.scala.relational.WebLogAnalysis;
-import org.apache.flink.test.testdata.WebLogAnalysisData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class WebLogAnalysisITCase extends JavaProgramTestBase {
-
-	private String docsPath;
-	private String ranksPath;
-	private String visitsPath;
-	private String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		docsPath = createTempFile("docs", WebLogAnalysisData.DOCS);
-		ranksPath = createTempFile("ranks", WebLogAnalysisData.RANKS);
-		visitsPath = createTempFile("visits", WebLogAnalysisData.VISITS);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WebLogAnalysisData.EXCEPTED_RESULT, resultPath);
-	}
-	@Override
-	protected void testProgram() throws Exception {
-		WebLogAnalysis.main(new String[]{
-				"--documents", docsPath,
-				"--ranks", ranksPath,
-				"--visits", visitsPath,
-				"--output", resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
deleted file mode 100644
index ce55910..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/exampleScalaPrograms/WordCountITCase.java
+++ /dev/null
@@ -1,53 +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.exampleScalaPrograms;
-
-import org.apache.flink.examples.scala.wordcount.WordCount;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class WordCountITCase extends JavaProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	public WordCountITCase(){
-		setParallelism(4);
-		setNumTaskManagers(2);
-		setTaskManagerNumSlots(2);
-	}
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WordCount.main(new String[] {
-				"--input", textPath,
-				"--output", resultPath });
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
deleted file mode 100644
index 10e3cb6..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.failingPrograms;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.TaskManagerOptions;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.client.JobSubmissionException;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testtasks.NoOpInvokable;
-
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-@RunWith(Parameterized.class)
-public class JobSubmissionFailsITCase extends TestLogger {
-	
-	private static final int NUM_SLOTS = 20;
-	
-	private static LocalFlinkMiniCluster cluster;
-	private static JobGraph workingJobGraph;
-
-	@BeforeClass
-	public static void setup() {
-		try {
-			Configuration config = new Configuration();
-			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L);
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2);
-			
-			cluster = new LocalFlinkMiniCluster(config);
-
-			cluster.start();
-			
-			final JobVertex jobVertex = new JobVertex("Working job vertex.");
-			jobVertex.setInvokableClass(NoOpInvokable.class);
-			workingJobGraph = new JobGraph("Working testing job", jobVertex);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void teardown() {
-		try {
-			cluster.shutdown();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	private boolean detached;
-
-	public JobSubmissionFailsITCase(boolean detached) {
-		this.detached = detached;
-	}
-
-	@Parameterized.Parameters(name = "Detached mode = {0}")
-	public static Collection<Boolean[]> executionModes(){
-		return Arrays.asList(new Boolean[]{false},
-				new Boolean[]{true});
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	private JobExecutionResult submitJob(JobGraph jobGraph) throws Exception {
-		if (detached) {
-			cluster.submitJobDetached(jobGraph);
-			return null;
-		}
-		else {
-			return cluster.submitJobAndWait(jobGraph, false, TestingUtils.TESTING_DURATION());
-		}
-	}
-
-	@Test
-	public void testExceptionInInitializeOnMaster() {
-		try {
-			final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex");
-			failingJobVertex.setInvokableClass(NoOpInvokable.class);
-
-			final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex);
-
-			try {
-				submitJob(failingJobGraph);
-				fail("Expected JobExecutionException.");
-			}
-			catch (JobExecutionException e) {
-				assertEquals("Test exception.", e.getCause().getMessage());
-			}
-			catch (Throwable t) {
-				t.printStackTrace();
-				fail("Caught wrong exception of type " + t.getClass() + ".");
-			}
-
-			cluster.submitJobAndWait(workingJobGraph, false);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSubmitEmptyJobGraph() {
-		try {
-			final JobGraph jobGraph = new JobGraph("Testing job");
-	
-			try {
-				submitJob(jobGraph);
-				fail("Expected JobSubmissionException.");
-			}
-			catch (JobSubmissionException e) {
-				assertTrue(e.getMessage() != null && e.getMessage().contains("empty"));
-			}
-			catch (Throwable t) {
-				t.printStackTrace();
-				fail("Caught wrong exception of type " + t.getClass() + ".");
-			}
-	
-			cluster.submitJobAndWait(workingJobGraph, false);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSubmitNullJobGraph() {
-		try {
-			try {
-				submitJob(null);
-				fail("Expected JobSubmissionException.");
-			}
-			catch (NullPointerException e) {
-				// yo!
-			}
-			catch (Throwable t) {
-				t.printStackTrace();
-				fail("Caught wrong exception of type " + t.getClass() + ".");
-			}
-
-			cluster.submitJobAndWait(workingJobGraph, false);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	public static class FailingJobVertex extends JobVertex {
-		private static final long serialVersionUID = -6365291240199412135L;
-
-		public FailingJobVertex(final String msg) {
-			super(msg);
-		}
-
-		@Override
-		public void initializeOnMaster(ClassLoader loader) throws Exception {
-			throw new Exception("Test exception.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
deleted file mode 100644
index f6fb0dd..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/TaskFailureITCase.java
+++ /dev/null
@@ -1,98 +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.failingPrograms;
-
-import java.util.List;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Assert;
-
-/**
- *
- * Tests that both jobs, the failing and the working one, are handled correctly. The first (failing) job must be
- * canceled and the client must report the failure. The second (working) job must finish successfully and compute the
- * correct result.
- *
- */
-public class TaskFailureITCase extends JavaProgramTestBase {
-
-	private static String EXCEPTION_STRING = "This is an expected Test Exception";
-
-	@Override
-	protected void testProgram() throws Exception {
-		//test failing version
-		try {
-			executeTask(new FailingTestMapper(), 1);
-		} catch (RuntimeException e) { //expected for collection execution
-			if (!isCollectionExecution()) {
-				Assert.fail();
-			}
-			// for collection execution, no restarts. So, exception should be appended with 0.
-			Assert.assertEquals(EXCEPTION_STRING + ":0", e.getMessage());
-		} catch (JobExecutionException e) { //expected for cluster execution
-			if (isCollectionExecution()) {
-				Assert.fail();
-			}
-			// for cluster execution, one restart. So, exception should be appended with 1.
-			Assert.assertEquals(EXCEPTION_STRING + ":1", e.getCause().getMessage());
-		}
-		//test correct version
-		executeTask(new TestMapper(), 0);
-	}
-	
-
-	private void executeTask(MapFunction<Long, Long> mapper, int retries) throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(retries, 0));
-		List<Long> result = env.generateSequence(1, 9)
-				.map(mapper)
-				.collect();
-		MultipleProgramsTestBase.compareResultAsText(result, "1\n2\n3\n4\n5\n6\n7\n8\n9");
-	}
-
-
-	/**
-	 * working map function
-	 */
-	public static class TestMapper implements MapFunction<Long, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long map(Long value) throws Exception {
-			return value;
-		}
-	}
-	
-	/**
-	 * failing map function
-	 */
-	public static class FailingTestMapper extends RichMapFunction<Long, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long map(Long value) throws Exception {
-			throw new RuntimeException(EXCEPTION_STRING + ":" + getRuntimeContext().getAttemptNumber());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/HadoopIOFormatsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/HadoopIOFormatsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/HadoopIOFormatsITCase.java
index 468b780..07b4d76 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/HadoopIOFormatsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/HadoopIOFormatsITCase.java
@@ -21,12 +21,13 @@ package org.apache.flink.test.hadoop.mapred;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.OperatingSystem;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -49,10 +50,13 @@ import java.net.URI;
 import java.util.Collection;
 import java.util.LinkedList;
 
+/**
+ * Integraiton tests for Hadoop IO formats.
+ */
 @RunWith(Parameterized.class)
 public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 
-	private static int NUM_PROGRAMS = 2;
+	private static final int NUM_PROGRAMS = 2;
 
 	private int curProgId = config.getInteger("ProgramId", -1);
 	private String[] resultPath;
@@ -61,9 +65,9 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 	private String sequenceFileInPathNull;
 
 	public HadoopIOFormatsITCase(Configuration config) {
-		super(config);	
+		super(config);
 	}
-	
+
 	@Before
 	public void checkOperatingSystem() {
 		// FLINK-5164 - see https://wiki.apache.org/hadoop/WindowsProblems
@@ -89,25 +93,24 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 		Text value = new Text();
 		SequenceFile.Writer writer = null;
 		try {
-			writer = SequenceFile.createWriter( fs, conf, path, key.getClass(), value.getClass());
-			for (int i = 0; i < kvCount; i ++) {
-				if(i == 1) {
+			writer = SequenceFile.createWriter(fs, conf, path, key.getClass(), value.getClass());
+			for (int i = 0; i < kvCount; i++) {
+				if (i == 1) {
 					// write key = 0 a bit more often.
-					for(int a = 0;a < 15; a++) {
+					for (int a = 0; a < 15; a++) {
 						key.set(i);
-						value.set(i+" - somestring");
+						value.set(i + " - somestring");
 						writer.append(key, value);
 					}
 				}
 				key.set(i);
-				value.set(i+" - somestring");
+				value.set(i + " - somestring");
 				writer.append(key, value);
 			}
 		} finally {
 			IOUtils.closeStream(writer);
 		}
 
-
 		//  ------------------ Long / Text Key Value pair: ------------
 
 		File sequenceFileNull = createAndRegisterTempFile("seqFileNullKey");
@@ -117,8 +120,8 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 		LongWritable value1 = new LongWritable();
 		SequenceFile.Writer writer1 = null;
 		try {
-			writer1 = SequenceFile.createWriter( fs, conf, path, NullWritable.class, value1.getClass());
-			for (int i = 0; i < kvCount; i ++) {
+			writer1 = SequenceFile.createWriter(fs, conf, path, NullWritable.class, value1.getClass());
+			for (int i = 0; i < kvCount; i++) {
 				value1.set(i);
 				writer1.append(NullWritable.get(), value1);
 			}
@@ -131,32 +134,32 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 	protected void testProgram() throws Exception {
 		expectedResult = HadoopIOFormatPrograms.runProgram(curProgId, resultPath, sequenceFileInPath, sequenceFileInPathNull);
 	}
-	
+
 	@Override
 	protected void postSubmit() throws Exception {
-		for(int i = 0; i < resultPath.length; i++) {
+		for (int i = 0; i < resultPath.length; i++) {
 			compareResultsByLinesInMemory(expectedResult[i], resultPath[i]);
 		}
 	}
-	
+
 	@Parameters
 	public static Collection<Object[]> getConfigurations() throws FileNotFoundException, IOException {
 
 		LinkedList<Configuration> tConfigs = new LinkedList<Configuration>();
 
-		for(int i=1; i <= NUM_PROGRAMS; i++) {
+		for (int i = 1; i <= NUM_PROGRAMS; i++) {
 			Configuration config = new Configuration();
 			config.setInteger("ProgramId", i);
 			tConfigs.add(config);
 		}
-		
+
 		return TestBaseUtils.toParameterList(tConfigs);
 	}
-	
-	public static class HadoopIOFormatPrograms {
-		
-		public static String[] runProgram(int progId, String resultPath[], String sequenceFileInPath, String sequenceFileInPathNull) throws Exception {
-			
+
+	private static class HadoopIOFormatPrograms {
+
+		public static String[] runProgram(int progId, String[] resultPath, String sequenceFileInPath, String sequenceFileInPathNull) throws Exception {
+
 			switch(progId) {
 			case 1: {
 				/**
@@ -184,7 +187,7 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 				});
 				res.writeAsText(resultPath[1]);
 				env.execute();
-				
+
 				// return expected result
 				return 	new String [] {"(21,3 - somestring)", "0 - somestring - 0\n" +
 						"1 - somestring - 1\n" +
@@ -224,8 +227,8 @@ public class HadoopIOFormatsITCase extends JavaProgramTestBase {
 			default:
 				throw new IllegalArgumentException("Invalid program id");
 			}
-			
+
 		}
-	
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/WordCountMapredITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/WordCountMapredITCase.java b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/WordCountMapredITCase.java
index 7c1b30e..21f6985 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/WordCountMapredITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapred/WordCountMapredITCase.java
@@ -23,11 +23,11 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
-import static org.apache.flink.hadoopcompatibility.HadoopInputs.readHadoopFile;
 import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.test.testfunctions.Tokenizer;
+import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.OperatingSystem;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
@@ -37,6 +37,11 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.junit.Assume;
 import org.junit.Before;
 
+import static org.apache.flink.hadoopcompatibility.HadoopInputs.readHadoopFile;
+
+/**
+ * Test WordCount with Hadoop input and output "mapred" (legacy) formats.
+ */
 public class WordCountMapredITCase extends JavaProgramTestBase {
 
 	protected String textPath;
@@ -87,7 +92,6 @@ public class WordCountMapredITCase extends JavaProgramTestBase {
 			}
 		});
 
-
 		DataSet<Tuple2<String, Integer>> counts =
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				text.flatMap(new Tokenizer())
@@ -97,7 +101,6 @@ public class WordCountMapredITCase extends JavaProgramTestBase {
 
 		DataSet<Tuple2<Text, LongWritable>> words = counts.map(new MapFunction<Tuple2<String, Integer>, Tuple2<Text, LongWritable>>() {
 
-
 			@Override
 			public Tuple2<Text, LongWritable> map(Tuple2<String, Integer> value) throws Exception {
 				return new Tuple2<Text, LongWritable>(new Text(value.f0), new LongWritable(value.f1));

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapreduce/WordCountMapreduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapreduce/WordCountMapreduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapreduce/WordCountMapreduceITCase.java
index fbc1994..0092fe7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapreduce/WordCountMapreduceITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/hadoop/mapreduce/WordCountMapreduceITCase.java
@@ -23,11 +23,11 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
-import static org.apache.flink.hadoopcompatibility.HadoopInputs.readHadoopFile;
 import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.test.testfunctions.Tokenizer;
+import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.OperatingSystem;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
@@ -37,6 +37,11 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.junit.Assume;
 import org.junit.Before;
 
+import static org.apache.flink.hadoopcompatibility.HadoopInputs.readHadoopFile;
+
+/**
+ * Test WordCount with Hadoop input and output "mapreduce" (modern) formats.
+ */
 public class WordCountMapreduceITCase extends JavaProgramTestBase {
 
 	protected String textPath;
@@ -86,7 +91,6 @@ public class WordCountMapreduceITCase extends JavaProgramTestBase {
 			}
 		});
 
-
 		DataSet<Tuple2<String, Integer>> counts =
 				// split up the lines in pairs (2-tuples) containing: (word,1)
 				text.flatMap(new Tokenizer())
@@ -96,7 +100,6 @@ public class WordCountMapreduceITCase extends JavaProgramTestBase {
 
 		DataSet<Tuple2<Text, LongWritable>> words = counts.map(new MapFunction<Tuple2<String, Integer>, Tuple2<Text, LongWritable>>() {
 
-
 			@Override
 			public Tuple2<Text, LongWritable> map(Tuple2<String, Integer> value) throws Exception {
 				return new Tuple2<Text, LongWritable>(new Text(value.f0), new LongWritable(value.f1));

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
index e50dfd3..82f699d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/io/CsvReaderITCase.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.test.io;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple8;
@@ -32,6 +30,9 @@ import org.apache.flink.types.IntValue;
 import org.apache.flink.types.LongValue;
 import org.apache.flink.types.ShortValue;
 import org.apache.flink.types.StringValue;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -42,6 +43,9 @@ import java.io.File;
 import java.util.List;
 import java.util.Locale;
 
+/**
+ * Tests for {@link ExecutionEnvironment#readCsvFile}.
+ */
 @RunWith(Parameterized.class)
 public class CsvReaderITCase extends MultipleProgramsTestBase {
 	private String expected;
@@ -122,6 +126,9 @@ public class CsvReaderITCase extends MultipleProgramsTestBase {
 		compareResultAsTuples(result, expected);
 	}
 
+	/**
+	 * POJO.
+	 */
 	public static class POJOItem {
 		public String f1;
 		private int f2;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java
index cfdc31a..90611eb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/io/InputOutputITCase.java
@@ -38,7 +38,7 @@ public class InputOutputITCase extends JavaProgramTestBase {
 		env.createInput(new TestNonRichInputFormat()).output(output);
 		try {
 			env.execute();
-		} catch(Exception e){
+		} catch (Exception e){
 			// we didn't break anything by making everything rich.
 			e.printStackTrace();
 			fail(e.getMessage());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java
index 98f19ff..4f25bad 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/io/RichInputOutputITCase.java
@@ -79,9 +79,9 @@ public class RichInputOutputITCase extends JavaProgramTestBase {
 
 		@Override
 		public void open(FileInputSplit split) throws IOException{
-			try{
+			try {
 				getRuntimeContext().addAccumulator("DATA_SOURCE_ACCUMULATOR", counter);
-			} catch(UnsupportedOperationException e){
+			} catch (UnsupportedOperationException e){
 				// the accumulator is already added
 			}
 			super.open(split);
@@ -104,9 +104,9 @@ public class RichInputOutputITCase extends JavaProgramTestBase {
 
 		@Override
 		public void open(int a, int b){
-			try{
+			try {
 				getRuntimeContext().addAccumulator("DATA_SINK_ACCUMULATOR", counter);
-			} catch(UnsupportedOperationException e){
+			} catch (UnsupportedOperationException e){
 				// the accumulator is already added
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/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 d55a63f..66adbde 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
@@ -18,50 +18,49 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.test.util.JavaProgramTestBase;
 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;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-
+import java.util.ArrayList;
+import java.util.List;
 
+/**
+ * Integration test for a bulk iteration with an all reduce.
+ */
 @SuppressWarnings("serial")
 public class BulkIterationWithAllReducerITCase extends JavaProgramTestBase {
 
 	@Override
 	protected void testProgram() throws Exception {
-		
+
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		DataSet<Integer> data = env.fromElements(1, 2, 3, 4, 5, 6, 7, 8);
-		
+
 		IterativeDataSet<Integer> iteration = data.iterate(10);
-		
+
 		DataSet<Integer> result = data.reduceGroup(new PickOneAllReduce()).withBroadcastSet(iteration, "bc");
-		
+
 		final List<Integer> resultList = new ArrayList<Integer>();
 		iteration.closeWith(result).output(new LocalCollectionOutputFormat<Integer>(resultList));
-		
+
 		env.execute();
-		
+
 		Assert.assertEquals(8, resultList.get(0).intValue());
 	}
 
-	
-	public static class PickOneAllReduce extends RichGroupReduceFunction<Integer, Integer> {
-		
+	private static class PickOneAllReduce extends RichGroupReduceFunction<Integer, Integer> {
+
 		private Integer bcValue;
-		
+
 		@Override
 		public void open(Configuration parameters) {
 			List<Integer> bc = getRuntimeContext().getBroadcastVariable("bc");
@@ -74,8 +73,8 @@ public class BulkIterationWithAllReducerITCase extends JavaProgramTestBase {
 				return;
 			}
 			final int x = bcValue;
-			
-			for (Integer y : records) { 
+
+			for (Integer y : records) {
 				if (y > x) {
 					out.collect(y);
 					return;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
index 2a0b004..2b53249 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsITCase.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import java.io.BufferedReader;
-import java.util.Iterator;
-
 import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.functions.MapFunction;
@@ -37,22 +34,26 @@ import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 
+import java.io.BufferedReader;
+import java.util.Iterator;
+
+/**
+ * Delta iteration test implementing the connected components algorithm with a cogroup.
+ */
 public class CoGroupConnectedComponentsITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	private static final int MAX_ITERATIONS = 100;
 
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
-	
-	
+
 	@Override
 	protected void preSubmit() throws Exception {
 		verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES));
@@ -66,11 +67,11 @@ public class CoGroupConnectedComponentsITCase extends JavaProgramTestBase {
 			ConnectedComponentsData.checkOddEvenResult(reader);
 		}
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	//  The test program
 	// --------------------------------------------------------------------------------------------
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 
@@ -111,7 +112,7 @@ public class CoGroupConnectedComponentsITCase extends JavaProgramTestBase {
 
 	@ForwardedFieldsFirst("f1->f1")
 	@ForwardedFieldsSecond("f0->f0")
-	public static final class MinIdAndUpdate implements CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static final class MinIdAndUpdate implements CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
index 7f5d194..a25cf24 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/CoGroupConnectedComponentsSecondITCase.java
@@ -18,56 +18,58 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.RichCoGroupFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
 import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
+import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.graph.ConnectedComponents.DuplicateValue;
+import org.apache.flink.examples.java.graph.ConnectedComponents.NeighborWithComponentIDJoin;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.examples.java.graph.ConnectedComponents.DuplicateValue;
-import org.apache.flink.examples.java.graph.ConnectedComponents.NeighborWithComponentIDJoin;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Delta iteration test implementing the connected components algorithm with a cogroup.
+ */
 @SuppressWarnings("serial")
 public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	@Override
 	protected void testProgram() throws Exception {
-			
+
 		// set up execution environment
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		// read vertex and edge data
 		DataSet<Long> vertices = env.fromElements(ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES).split("\n"))
 				.map(new VertexParser());
-		
+
 		DataSet<Tuple2<Long, Long>> edges = env.fromElements(ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED).split("\n"))
 				.flatMap(new EdgeParser());
-		
+
 		// assign the initial components (equal to the vertex id)
 		DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new DuplicateValue<Long>());
-				
+
 		// open a delta iteration
 		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
 				verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0);
-		
+
 		// apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller
 		DataSet<Tuple2<Long, Long>> changes = iteration
 				.getWorkset().join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
@@ -76,35 +78,34 @@ public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBase
 
 		// close the delta iteration (delta and new workset are identical)
 		DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
-		
-		
+
 		// emit result
-		List<Tuple2<Long,Long>> resutTuples = new ArrayList<Tuple2<Long,Long>>();
-		result.output(new LocalCollectionOutputFormat<Tuple2<Long,Long>>(resutTuples));
-		
+		List<Tuple2<Long, Long>> resutTuples = new ArrayList<>();
+		result.output(new LocalCollectionOutputFormat<>(resutTuples));
+
 		env.execute();
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	//  The test program
 	// --------------------------------------------------------------------------------------------
-	
-	public static final class VertexParser extends RichMapFunction<String, Long> {
+
+	private static final class VertexParser extends RichMapFunction<String, Long> {
 
 		@Override
 		public Long map(String value) throws Exception {
 			return Long.parseLong(value);
 		}
 	}
-	
-	public static final class EdgeParser extends RichFlatMapFunction<String, Tuple2<Long, Long>> {
+
+	private static final class EdgeParser extends RichFlatMapFunction<String, Tuple2<Long, Long>> {
 
 		@Override
 		public void flatMap(String value, Collector<Tuple2<Long, Long>> out) throws Exception {
 			String[] parts = value.split(" ");
 			long v1 = Long.parseLong(parts[0]);
 			long v2 = Long.parseLong(parts[1]);
-			
+
 			out.collect(new Tuple2<Long, Long>(v1, v2));
 			out.collect(new Tuple2<Long, Long>(v2, v1));
 		}
@@ -112,17 +113,17 @@ public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBase
 
 	@ForwardedFieldsFirst("0")
 	@ForwardedFieldsSecond("0")
-	public static final class MinIdAndUpdate extends RichCoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		
+	private static final class MinIdAndUpdate extends RichCoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
 		@Override
 		public void coGroup(Iterable<Tuple2<Long, Long>> candidates, Iterable<Tuple2<Long, Long>> current, Collector<Tuple2<Long, Long>> out) {
 			Iterator<Tuple2<Long, Long>> iterator = current.iterator();
 			if (!iterator.hasNext()) {
 				throw new RuntimeException("Error: Id not encountered before.");
 			}
-			
+
 			Tuple2<Long, Long> old = iterator.next();
-			
+
 			long minimumComponentID = Long.MAX_VALUE;
 
 			for (Tuple2<Long, Long> candidate : candidates) {
@@ -131,7 +132,7 @@ public class CoGroupConnectedComponentsSecondITCase extends JavaProgramTestBase
 					minimumComponentID = candidateComponentID;
 				}
 			}
-			
+
 			if (minimumComponentID < old.f1) {
 				old.f1 = minimumComponentID;
 				out.collect(old);

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
index 61e08d4..ee5597b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsITCase.java
@@ -16,11 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative;
 
-import java.io.BufferedReader;
-
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -32,16 +29,19 @@ import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.io.BufferedReader;
 
+/**
+ * Delta iteration test implementing the connected components algorithm with a join.
+ */
 public class ConnectedComponentsITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
@@ -93,6 +93,11 @@ public class ConnectedComponentsITCase extends JavaProgramTestBase {
 		}
 	}
 
+	/**
+	 * Duplicate the vertex ID into both fields of a {@link Tuple2}.
+	 *
+	 * @param <T> key type
+	 */
 	public static final class DuplicateValue<T> implements MapFunction<Tuple1<T>, Tuple2<T, T>> {
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
index bc4885f..e425f29 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithDeferredUpdateITCase.java
@@ -16,12 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative;
 
-import java.io.BufferedReader;
-import java.util.Collection;
-
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
@@ -35,25 +31,31 @@ import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import java.io.BufferedReader;
+import java.util.Collection;
+
+/**
+ * Delta iteration test implementing the connected components algorithm with a
+ * cogroup and join on the solution set.
+ */
 @RunWith(Parameterized.class)
 public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
-	
-	
+
 	public ConnectedComponentsWithDeferredUpdateITCase(Configuration config) {
 		super(config);
 	}
@@ -92,8 +94,8 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTest
 				.join(iteration.getSolutionSet()).where(0).equalTo(0)
 				.with(new UpdateComponentIdMatchNonPreserving());
 
-		DataSet<Tuple2<Long,Long>> delta;
-		if(extraMapper) {
+		DataSet<Tuple2<Long, Long>> delta;
+		if (extraMapper) {
 			delta = changes.map(
 					// ID Mapper
 					new MapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>() {
@@ -127,14 +129,14 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTest
 	public static Collection<Object[]> getConfigurations() {
 		Configuration config1 = new Configuration();
 		config1.setBoolean("ExtraMapper", false);
-		
+
 		Configuration config2 = new Configuration();
 		config2.setBoolean("ExtraMapper", true);
-		
+
 		return toParameterList(config1, config2);
 	}
 
-	public static final class UpdateComponentIdMatchNonPreserving
+	private static final class UpdateComponentIdMatchNonPreserving
 			implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
 
@@ -144,7 +146,7 @@ public class ConnectedComponentsWithDeferredUpdateITCase extends JavaProgramTest
 				Tuple2<Long, Long> current,
 				Collector<Tuple2<Long, Long>> out) throws Exception {
 
-			if(candidate.f1 < current.f1) {
+			if (candidate.f1 < current.f1) {
 				out.collect(candidate);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java
index a8a28f1..ce88ab6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithObjectMapITCase.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import java.io.BufferedReader;
-
-import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.aggregation.Aggregations;
@@ -33,22 +30,24 @@ import org.apache.flink.examples.java.graph.ConnectedComponents.UndirectEdge;
 import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.io.BufferedReader;
+
+/**
+ * Delta iteration test implementing the connected components algorithm with an object map.
+ */
 @SuppressWarnings("serial")
 public class ConnectedComponentsWithObjectMapITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
 
-	
-	
 	@Override
 	protected void preSubmit() throws Exception {
 		verticesPath = createTempFile("vertices.txt", ConnectedComponentsData.getEnumeratingVertices(NUM_VERTICES));
@@ -63,26 +62,25 @@ public class ConnectedComponentsWithObjectMapITCase extends JavaProgramTestBase
 		}
 	}
 
-	
 	@Override
 	protected void testProgram() throws Exception {
 		// set up execution environment
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-				
+
 		// read vertex and edge data
 		DataSet<Tuple1<Long>> vertices = env.readCsvFile(verticesPath).types(Long.class);
-		
+
 		DataSet<Tuple2<Long, Long>> edges = env.readCsvFile(edgesPath).fieldDelimiter(" ").types(Long.class, Long.class)
 												.flatMap(new UndirectEdge());
-				
+
 		// assign the initial components (equal to the vertex id)
 		DataSet<Tuple2<Long, Long>> verticesWithInitialId = vertices.map(new ConnectedComponentsITCase.DuplicateValue<Long>());
-						
+
 		// open a delta iteration
 		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
 				verticesWithInitialId.iterateDelta(verticesWithInitialId, 100, 0);
 		iteration.setSolutionSetUnManaged(true);
-				
+
 		// apply the step logic: join with the edges, select the minimum neighbor, update if the component of the candidate is smaller
 		DataSet<Tuple2<Long, Long>> changes = iteration.getWorkset().join(edges).where(0).equalTo(0).with(new NeighborWithComponentIDJoin())
 				.groupBy(0).aggregate(Aggregations.MIN, 1)
@@ -91,11 +89,11 @@ public class ConnectedComponentsWithObjectMapITCase extends JavaProgramTestBase
 
 		// close the delta iteration (delta and new workset are identical)
 		DataSet<Tuple2<Long, Long>> result = iteration.closeWith(changes, changes);
-				
+
 		result.writeAsCsv(resultPath, "\n", " ");
-		
+
 		// execute program
 		env.execute("Connected Components Example");
 	}
-	
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
index c2dd434..6f35917 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/ConnectedComponentsWithSolutionSetFirstITCase.java
@@ -16,11 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative;
 
-import java.io.BufferedReader;
-
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -34,18 +31,19 @@ import org.apache.flink.test.testdata.ConnectedComponentsData;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
 
+import java.io.BufferedReader;
+
 /**
  * Tests a bug that prevented that the solution set can be on both sides of the match/cogroup function.
  */
 public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTestBase {
-	
+
 	private static final long SEED = 0xBADC0FFEEBEEFL;
-	
+
 	private static final int NUM_VERTICES = 1000;
-	
+
 	private static final int NUM_EDGES = 10000;
 
-	
 	protected String verticesPath;
 	protected String edgesPath;
 	protected String resultPath;
@@ -56,7 +54,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTe
 		edgesPath = createTempFile("edges.txt", ConnectedComponentsData.getRandomOddEvenEdges(NUM_EDGES, NUM_VERTICES, SEED));
 		resultPath = getTempFilePath("results");
 	}
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		// set up execution environment
@@ -92,7 +90,6 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTe
 		env.execute("Connected Components Example");
 	}
 
-
 	@Override
 	protected void postSubmit() throws Exception {
 		for (BufferedReader reader : getResultReader(resultPath)) {
@@ -105,7 +102,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTe
 	// --------------------------------------------------------------------------------------------
 
 	@FunctionAnnotation.ForwardedFieldsSecond("*")
-	public static final class UpdateComponentIdMatchMirrored
+	private static final class UpdateComponentIdMatchMirrored
 			implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
 
@@ -115,7 +112,7 @@ public class ConnectedComponentsWithSolutionSetFirstITCase extends JavaProgramTe
 				Tuple2<Long, Long> candidate,
 				Collector<Tuple2<Long, Long>> out) throws Exception {
 
-			if(candidate.f1 < current.f1) {
+			if (candidate.f1 < current.f1) {
 				out.collect(candidate);
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/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 53496e2..8f8f28b 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
@@ -46,14 +46,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-
+/**
+ * Implementation of PageRank accounting for "sink" vertices with 0 out-degree.
+ */
 @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);
 	}
@@ -61,9 +62,9 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 	@Test
 	public void testDanglingPageRank() {
 		try {
-			final int NUM_ITERATIONS = 25;
+			final int numIterations = 25;
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
+
 			DataSet<Tuple2<Long, Boolean>> vertices = env.fromElements(
 					new Tuple2<>(1L, false),
 					new Tuple2<>(2L, false),
@@ -78,8 +79,7 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 					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(
@@ -90,32 +90,31 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 								}
 							})
 					.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);
+
+			IterativeDataSet<PageWithRankAndDangling> iteration = verticesWithInitialRank.iterate(numIterations);
 
 			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);
 
@@ -126,8 +125,8 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 						}
 					}
 			);
-			
-			DataSet<PageWithRankAndDangling> newRanks = 
+
+			DataSet<PageWithRankAndDangling> newRanks =
 				iteration.coGroup(partialRanks).where("pageId").equalTo("pageId").with(
 					new RichCoGroupFunction<PageWithRankAndDangling, PageWithRank, PageWithRankAndDangling>() {
 
@@ -136,15 +135,15 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 						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 / 
+								danglingRankFactor = BETA * (double) numDanglingVertices /
 										((double) numVertices * (double) numVertices);
 							} else {
 								PageRankStats previousAggregate = getIterationRuntimeContext()
@@ -152,12 +151,12 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 								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;
@@ -166,7 +165,7 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 								edges++;
 							}
 							double rank = BETA * summedRank + randomJump + danglingRankFactor;
-							
+
 							// current rank, for stats and convergence
 							PageWithRankAndDangling currentPage = currentPages.iterator().next();
 							double currentRank = currentPage.rank;
@@ -182,16 +181,16 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 							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) {
@@ -203,25 +202,31 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 	// ------------------------------------------------------------------------
 	//  custom types
 	// ------------------------------------------------------------------------
-	
+
+	/**
+	 * POJO for page ID and rank value.
+	 */
 	public static class PageWithRank {
-		
+
 		public long pageId;
 		public double rank;
 
 		public PageWithRank() {}
-		
+
 		public PageWithRank(long pageId, double rank) {
 			this.pageId = pageId;
 			this.rank = rank;
 		}
 	}
 
+	/**
+	 * POJO for page ID, rank value, and whether a "dangling" vertex with 0 out-degree.
+	 */
 	public static class PageWithRankAndDangling {
 
 		public long pageId;
 		public double rank;
-		public boolean dangling; 
+		public boolean dangling;
 
 		public PageWithRankAndDangling() {}
 
@@ -241,6 +246,9 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 		}
 	}
 
+	/**
+	 * POJO for page ID and list of target IDs.
+	 */
 	public static class PageWithLinks {
 
 		public long pageId;
@@ -253,11 +261,14 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 			this.targets = targets;
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  statistics
 	// ------------------------------------------------------------------------
 
+	/**
+	 * PageRank statistics.
+	 */
 	public static class PageRankStats implements Value {
 
 		private double diff;
@@ -272,7 +283,7 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 		public PageRankStats(
 					double diff, double rank, double danglingRank,
 					long numDanglingVertices, long numVertices, long edges) {
-			
+
 			this.diff = diff;
 			this.rank = rank;
 			this.danglingRank = danglingRank;
@@ -332,8 +343,8 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 					"]";
 		}
 	}
-	
-	public static class PageRankStatsAggregator implements Aggregator<PageRankStats> {
+
+	private static class PageRankStatsAggregator implements Aggregator<PageRankStats> {
 
 		private double diff;
 		private double rank;
@@ -348,7 +359,7 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 		}
 
 		public void aggregate(double diffDelta, double rankDelta, double danglingRankDelta, long danglingVerticesDelta,
-							  long verticesDelta, long edgesDelta) {
+				long verticesDelta, long edgesDelta) {
 			diff += diffDelta;
 			rank += rankDelta;
 			danglingRank += danglingRankDelta;
@@ -378,7 +389,7 @@ public class DanglingPageRankITCase extends MultipleProgramsTestBase {
 		}
 	}
 
-	public static class DiffL1NormConvergenceCriterion implements ConvergenceCriterion<PageRankStats> {
+	private static class DiffL1NormConvergenceCriterion implements ConvergenceCriterion<PageRankStats> {
 
 		private static final double EPSILON = 0.00005;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
index caa9d37..84feacf 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DeltaIterationNotDependingOnSolutionSetITCase.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.DataSet;
@@ -32,9 +27,18 @@ import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test delta iterations that do not join with the solution set.
+ */
 @SuppressWarnings("serial")
 public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTestBase {
-	private final List<Tuple2<Long, Long>> result = new ArrayList<Tuple2<Long,Long>>();
+	private final List<Tuple2<Long, Long>> result = new ArrayList<>();
 
 	@Override
 	protected void testProgram() throws Exception {
@@ -47,7 +51,7 @@ public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTe
 			DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration = input.iterateDelta(input, 5, 1);
 
 			iteration.closeWith(iteration.getWorkset(), iteration.getWorkset().map(new TestMapper()))
-					.output(new LocalCollectionOutputFormat<Tuple2<Long,Long>>(result));
+					.output(new LocalCollectionOutputFormat<Tuple2<Long, Long>>(result));
 
 			env.execute();
 		}
@@ -75,11 +79,11 @@ public class DeltaIterationNotDependingOnSolutionSetITCase extends JavaProgramTe
 			return new Tuple2<T, T>(value, value);
 		}
 	}
-	
+
 	private static final class TestMapper extends RichMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		@Override
 		public Tuple2<Long, Long> map(Tuple2<Long, Long> value) {
-			return new Tuple2<Long, Long>(value.f0+10, value.f1+10);
+			return new Tuple2<>(value.f0 + 10, value.f1 + 10);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
index 0635fe5..74e3da2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DependencyConnectedComponentsITCase.java
@@ -18,30 +18,28 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.ArrayList;
-import java.util.List;
-
 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.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;
 import org.apache.flink.util.Collector;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.ExecutionEnvironment;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
- * 
  * Iterative Connected Components test case which recomputes only the elements
  * of the solution set whose at least one dependency (in-neighbor) has changed since the last iteration.
  * Requires two joins with the solution set.
- *
  */
 @SuppressWarnings("serial")
 public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
-	
+
 	private static final int MAX_ITERATIONS = 20;
 	private static final int parallelism = 1;
 
@@ -53,46 +51,45 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 	public DependencyConnectedComponentsITCase(){
 		setTaskManagerNumSlots(parallelism);
 	}
-	
-	
+
 	@Override
 	protected void preSubmit() throws Exception {
 		verticesInput.clear();
 		edgesInput.clear();
 
 		// vertices input
-		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));
-		
+		verticesInput.add(new Tuple2<>(1L, 1L));
+		verticesInput.add(new Tuple2<>(2L, 2L));
+		verticesInput.add(new Tuple2<>(3L, 3L));
+		verticesInput.add(new Tuple2<>(4L, 4L));
+		verticesInput.add(new Tuple2<>(5L, 5L));
+		verticesInput.add(new Tuple2<>(6L, 6L));
+		verticesInput.add(new Tuple2<>(7L, 7L));
+		verticesInput.add(new Tuple2<>(8L, 8L));
+		verticesInput.add(new Tuple2<>(9L, 9L));
+
 		// vertices input
-		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));
-		
+		edgesInput.add(new Tuple2<>(1L, 2L));
+		edgesInput.add(new Tuple2<>(1L, 3L));
+		edgesInput.add(new Tuple2<>(2L, 3L));
+		edgesInput.add(new Tuple2<>(2L, 4L));
+		edgesInput.add(new Tuple2<>(2L, 1L));
+		edgesInput.add(new Tuple2<>(3L, 1L));
+		edgesInput.add(new Tuple2<>(3L, 2L));
+		edgesInput.add(new Tuple2<>(4L, 2L));
+		edgesInput.add(new Tuple2<>(4L, 6L));
+		edgesInput.add(new Tuple2<>(5L, 6L));
+		edgesInput.add(new Tuple2<>(6L, 4L));
+		edgesInput.add(new Tuple2<>(6L, 5L));
+		edgesInput.add(new Tuple2<>(7L, 8L));
+		edgesInput.add(new Tuple2<>(7L, 9L));
+		edgesInput.add(new Tuple2<>(8L, 7L));
+		edgesInput.add(new Tuple2<>(8L, 9L));
+		edgesInput.add(new Tuple2<>(9L, 7L));
+		edgesInput.add(new Tuple2<>(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";
 	}
@@ -101,63 +98,67 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 	protected void testProgram() throws Exception {
 		DependencyConnectedComponentsProgram.runProgram(resultPath);
 	}
-	
+
 	@Override
 	protected void postSubmit() throws Exception {
 		compareResultsByLinesInMemory(expectedResult, resultPath);
 	}
 
-	
 	private static class DependencyConnectedComponentsProgram {
-		
+
 		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);
 			int keyPosition = 0;
-			
+
 			DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
 					initialSolutionSet.iterateDelta(initialSolutionSet, MAX_ITERATIONS, keyPosition);
-			
+
 			DataSet<Long> candidates = iteration.getWorkset().join(edges).where(0).equalTo(0)
 					.with(new FindCandidatesJoin())
-					.groupBy(new KeySelector<Long, Long>() { 
-                        public Long getKey(Long id) { return id; } 
-                      }).reduceGroup(new RemoveDuplicatesReduce());
-			
-			DataSet<Tuple2<Long, Long>> candidatesDependencies = 
+					.groupBy(new KeySelector<Long, Long>() {
+						public Long getKey(Long id) {
+							return id;
+						}
+					}).reduceGroup(new RemoveDuplicatesReduce());
+
+			DataSet<Tuple2<Long, Long>> candidatesDependencies =
 					candidates.join(edges)
-					.where(new KeySelector<Long, Long>() { 
-                        public Long getKey(Long id) { return id; } 
-                      }).equalTo(new KeySelector<Tuple2<Long, Long>, Long>() { 
-                        public Long getKey(Tuple2<Long, Long> vertexWithId) 
-                        { return vertexWithId.f1; } 
-                      }).with(new FindCandidatesDependenciesJoin());
-			
-			DataSet<Tuple2<Long, Long>> verticesWithNewComponents = 
+					.where(new KeySelector<Long, Long>() {
+						public Long getKey(Long id) {
+							return id;
+						}
+					}).equalTo(new KeySelector<Tuple2<Long, Long>, Long>() {
+						public Long getKey(Tuple2<Long, Long> vertexWithId) {
+							return vertexWithId.f1;
+						}
+					}).with(new FindCandidatesDependenciesJoin());
+
+			DataSet<Tuple2<Long, Long>> verticesWithNewComponents =
 					candidatesDependencies.join(iteration.getSolutionSet()).where(0).equalTo(0)
 					.with(new NeighborWithComponentIDJoin())
 					.groupBy(0).reduceGroup(new MinimumReduce());
-			
-			DataSet<Tuple2<Long, Long>> updatedComponentId = 
+
+			DataSet<Tuple2<Long, Long>> updatedComponentId =
 					verticesWithNewComponents.join(iteration.getSolutionSet()).where(0).equalTo(0)
 					.flatMap(new MinimumIdFilter());
-			
+
 			iteration.closeWith(updatedComponentId, updatedComponentId).writeAsText(resultPath);
-			
+
 			env.execute();
-			
+
 			return resultPath;
 		}
 	}
-	
-	public static final class FindCandidatesJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long> {
-		
+
+	private static final class FindCandidatesJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long> {
+
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Long join(Tuple2<Long, Long> vertexWithCompId,
 				Tuple2<Long, Long> edge) throws Exception {
@@ -165,9 +166,9 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 			return edge.f1;
 		}
 	}
-	
-	public static final class RemoveDuplicatesReduce extends RichGroupReduceFunction<Long, Long> {
-		
+
+	private static final class RemoveDuplicatesReduce extends RichGroupReduceFunction<Long, Long> {
+
 		private static final long serialVersionUID = 1L;
 
 		@Override
@@ -175,35 +176,35 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 				out.collect(values.iterator().next());
 		}
 	}
-	
-	public static final class FindCandidatesDependenciesJoin extends RichJoinFunction<Long, Tuple2<Long, Long>,Tuple2<Long, Long>> {
-	
+
+	private static final class FindCandidatesDependenciesJoin extends RichJoinFunction<Long, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Tuple2<Long, Long> join(Long candidateId, Tuple2<Long, Long> edge) throws Exception {
 			return edge;
 		}
 	}
-	
-	public static final class NeighborWithComponentIDJoin extends RichJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
-	
+
+	private 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> edge,
 				Tuple2<Long, Long> vertexWithCompId) 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 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;
@@ -223,16 +224,15 @@ public class DependencyConnectedComponentsITCase extends JavaProgramTestBase {
 		}
 	}
 
-	public static final class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
-	
+	private static final class MinimumIdFilter extends RichFlatMapFunction<Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>>, Tuple2<Long, Long>> {
+
 		private static final long serialVersionUID = 1L;
-	
+
 		@Override
 		public void flatMap(
 				Tuple2<Tuple2<Long, Long>, Tuple2<Long, Long>> vertexWithNewAndOldId,
-				Collector<Tuple2<Long, Long>> out)
-		{
-			if ( vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1 ) {
+				Collector<Tuple2<Long, Long>> out) {
+			if (vertexWithNewAndOldId.f0.f1 < vertexWithNewAndOldId.f1.f1) {
 				out.collect(vertexWithNewAndOldId.f0);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java
index 3f02064..f2548a9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/EmptyWorksetIterationITCase.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.test.iterative;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -29,31 +26,37 @@ import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test empty (identity) delta iteration.
+ */
 @SuppressWarnings("serial")
 public class EmptyWorksetIterationITCase extends JavaProgramTestBase {
-	
+
 	private List<Tuple2<Long, Long>> result = new ArrayList<Tuple2<Long, Long>>();
-	
+
 	@Override
 	protected void testProgram() throws Exception {
-		
+
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		DataSet<Tuple2<Long, Long>> input = env.generateSequence(1, 20).map(new Dupl());
-				
+
 		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iter = input.iterateDelta(input, 20, 0);
 		iter.closeWith(iter.getWorkset(), iter.getWorkset())
 			.output(new LocalCollectionOutputFormat<Tuple2<Long, Long>>(result));
-		
+
 		env.execute();
 	}
 
-	public static final class Dupl implements MapFunction<Long, Tuple2<Long, Long>> {
+	private static final class Dupl implements MapFunction<Long, Tuple2<Long, Long>> {
 
 		@Override
 		public Tuple2<Long, Long> map(Long value) {
 			return new Tuple2<Long, Long>(value, value);
 		}
-		
+
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java
index c422aa4..9e9b5ab 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IdentityIterationITCase.java
@@ -18,35 +18,38 @@
 
 package org.apache.flink.test.iterative;
 
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test empty (identity) bulk iteration.
+ */
 public class IdentityIterationITCase extends JavaProgramTestBase {
 
 	private List<Long> result = new ArrayList<Long>();
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		IterativeDataSet<Long> iteration = env.generateSequence(1, 10).iterate(100);
 		iteration.closeWith(iteration)
 			.output(new LocalCollectionOutputFormat<Long>(result));
-		
+
 		env.execute();
 	}
-	
+
 	@Override
 	protected void postSubmit()  {
 		assertEquals(10, result.size());
-		
+
 		long sum = 0;
 		for (Long l : result) {
 			sum += l;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java
index 48788a5..458e453 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteDynamicPathConsumptionITCase.java
@@ -26,16 +26,20 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+/**
+ * Test where the test data is constructed such that the merge join zig zag
+ * has an early out, leaving elements on the dynamic path input unconsumed.
+ */
 @SuppressWarnings("serial")
 public class IterationIncompleteDynamicPathConsumptionITCase extends JavaProgramTestBase {
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		
+
 		// the test data is constructed such that the merge join zig zag
 		// has an early out, leaving elements on the dynamic path input unconsumed
-		
+
 		DataSet<Path> edges = env.fromElements(
 				new Path(1, 2),
 				new Path(1, 4),
@@ -46,24 +50,24 @@ public class IterationIncompleteDynamicPathConsumptionITCase extends JavaProgram
 				new Path(3, 14),
 				new Path(3, 16),
 				new Path(1, 18),
-				new Path(1, 20) );
-		
+				new Path(1, 20));
+
 		IterativeDataSet<Path> currentPaths = edges.iterate(10);
-		
+
 		DataSet<Path> newPaths = currentPaths
 				.join(edges, JoinHint.REPARTITION_SORT_MERGE).where("to").equalTo("from")
 					.with(new PathConnector())
 				.union(currentPaths).distinct("from", "to");
-		
+
 		DataSet<Path> result = currentPaths.closeWith(newPaths);
-		
+
 		result.output(new DiscardingOutputFormat<Path>());
-		
+
 		env.execute();
 	}
-	
+
 	private static class PathConnector implements JoinFunction<Path, Path, Path> {
-		
+
 		@Override
 		public Path join(Path path, Path edge)  {
 			return new Path(path.from, edge.to);
@@ -71,19 +75,22 @@ public class IterationIncompleteDynamicPathConsumptionITCase extends JavaProgram
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * Simple POJO.
+	 */
 	public static class Path {
-		
+
 		public long from;
 		public long to;
-		
+
 		public Path() {}
-		
+
 		public Path(long from, long to) {
 			this.from = from;
 			this.to = to;
 		}
-		
+
 		@Override
 		public String toString() {
 			return "(" + from + "," + to + ")";

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java
index b42e86b..f7cd111 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/IterationIncompleteStaticPathConsumptionITCase.java
@@ -26,16 +26,20 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.test.util.JavaProgramTestBase;
 
+/**
+ * Test where the test data is constructed such that the merge join zig zag
+ * has an early out, leaving elements on the static path input unconsumed.
+ */
 @SuppressWarnings("serial")
 public class IterationIncompleteStaticPathConsumptionITCase extends JavaProgramTestBase {
-	
+
 	@Override
 	protected void testProgram() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-	
+
 		// the test data is constructed such that the merge join zig zag
 		// has an early out, leaving elements on the static path input unconsumed
-		
+
 		DataSet<Path> edges = env.fromElements(
 				new Path(2, 1),
 				new Path(4, 1),
@@ -46,24 +50,24 @@ public class IterationIncompleteStaticPathConsumptionITCase extends JavaProgramT
 				new Path(14, 3),
 				new Path(16, 3),
 				new Path(18, 1),
-				new Path(20, 1) );
-		
+				new Path(20, 1));
+
 		IterativeDataSet<Path> currentPaths = edges.iterate(10);
-		
+
 		DataSet<Path> newPaths = currentPaths
 				.join(edges, JoinHint.REPARTITION_SORT_MERGE).where("to").equalTo("from")
 					.with(new PathConnector())
 				.union(currentPaths).distinct("from", "to");
-		
+
 		DataSet<Path> result = currentPaths.closeWith(newPaths);
-		
+
 		result.output(new DiscardingOutputFormat<Path>());
-		
+
 		env.execute();
 	}
-	
+
 	private static class PathConnector implements JoinFunction<Path, Path, Path> {
-		
+
 		@Override
 		public Path join(Path path, Path edge)  {
 			return new Path(path.from, edge.to);
@@ -71,19 +75,22 @@ public class IterationIncompleteStaticPathConsumptionITCase extends JavaProgramT
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * Simple POJO.
+	 */
 	public static class Path {
-		
+
 		public long from;
 		public long to;
-		
+
 		public Path() {}
-		
+
 		public Path(long from, long to) {
 			this.from = from;
 			this.to = to;
 		}
-		
+
 		@Override
 		public String toString() {
 			return "(" + from + "," + to + ")";


[16/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
deleted file mode 100644
index 4ca0f3e..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CoGroupITCase.java
+++ /dev/null
@@ -1,992 +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.javaApiOperators;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.distributions.DataDistribution;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.RichCoGroupFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.utils.DataSetUtils;
-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.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class CoGroupITCase extends MultipleProgramsTestBase {
-
-	public CoGroupITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	/*
-	 * CoGroup on tuples with key field selector
-	 */
-	@Test
-	public void testCoGroupTuplesWithKeyFieldSelector() throws Exception {
-		
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<Integer, Integer>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5CoGroup());
-
-		List<Tuple2<Integer, Integer>> result = coGroupDs.collect();
-
-		String expected = "1,0\n" +
-				"2,6\n" +
-				"3,24\n" +
-				"4,60\n" +
-				"5,120\n";
-		
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupOnTwoCustomTypeInputsWithKeyExtractors() throws Exception {
-		/*
-		 * CoGroup on two custom type inputs with key extractors
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> coGroupDs = ds.coGroup(ds2).where(new KeySelector4()).equalTo(new
-				KeySelector5()).with(new CustomTypeCoGroup());
-
-		List<CustomType> result = coGroupDs.collect();
-
-		String expected = "1,0,test\n" +
-				"2,6,test\n" +
-				"3,24,test\n" +
-				"4,60,test\n" +
-				"5,120,test\n" +
-				"6,210,test\n";
-		
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector4 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	public static class KeySelector5 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfCoGroupIfUDFReturnsLeftInputObjects() throws Exception {
-		/*
-		 * check correctness of cogroup if UDF returns left input objects
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple3ReturnLeft());
-
-		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n";
-		
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCoGroupIfUDFReturnsRightInputObjects() throws Exception {
-		/*
-		 * check correctness of cogroup if UDF returns right input objects
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5ReturnRight());
-		
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = coGroupDs.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"3,4,3,Hallo Welt wie gehts?,2\n" +
-				"3,5,4,ABC,2\n" +
-				"3,6,5,BCD,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithBroadcastSet() throws Exception {
-		/*
-		 * Reduce with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Integer, Integer>> coGroupDs = ds.coGroup(ds2).where(0).equalTo(0).with(new Tuple5CoGroupBC()).withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<Integer, Integer, Integer>> result = coGroupDs.collect();
-
-		String expected = "1,0,55\n" +
-				"2,6,55\n" +
-				"3,24,55\n" +
-				"4,60,55\n" +
-				"5,120,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
-	throws Exception {
-		/*
-		 * CoGroup on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds.coGroup(ds2).where(2).equalTo(new
-				KeySelector2()).with(new MixedCoGroup());
-
-		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
-
-		String expected = "0,1,test\n" +
-				"1,2,test\n" +
-				"2,5,test\n" +
-				"3,15,test\n" +
-				"4,33,test\n" +
-				"5,63,test\n" +
-				"6,109,test\n" +
-				"7,4,test\n" +
-				"8,4,test\n" +
-				"9,4,test\n" +
-				"10,5,test\n" +
-				"11,5,test\n" +
-				"12,5,test\n" +
-				"13,5,test\n" +
-				"14,5,test\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCoGroupOnACustomTypeWithKeyExtractorAndATupleInputWithKeyFieldSelector()
-			throws Exception {
-		/*
-		 * CoGroup on a tuple input with key field selector and a custom type input with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> coGroupDs = ds2.coGroup(ds).where(new KeySelector3()).equalTo(2).with
-				(new MixedCoGroup2());
-
-		List<CustomType> result = coGroupDs.collect();
-
-		String expected = "0,1,test\n" +
-				"1,2,test\n" +
-				"2,5,test\n" +
-				"3,15,test\n" +
-				"4,33,test\n" +
-				"5,63,test\n" +
-				"6,109,test\n" +
-				"7,4,test\n" +
-				"8,4,test\n" +
-				"9,4,test\n" +
-				"10,5,test\n" +
-				"11,5,test\n" +
-				"12,5,test\n" +
-				"13,5,test\n" +
-				"14,5,test\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector3 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCoGroupWithMultipleKeyFieldsWithFieldSelector() throws Exception {
-		/*
-		 * CoGroup with multiple key fields
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
-				where(0, 4).equalTo(0, 1).with(new Tuple5Tuple3CoGroup());
-
-		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
-		
-		String expected = "1,1,Hallo\n" +
-				"2,2,Hallo Welt\n" +
-				"3,2,Hallo Welt wie gehts?\n" +
-				"3,2,ABC\n" +
-				"5,3,HIJ\n" +
-				"5,3,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithMultipleKeyFieldsWithStaticClassKeyExtractor() throws Exception {
-		/*
-		 * CoGroup with multiple key fields
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
-				where(new KeySelector7()).
-				equalTo(new KeySelector8()).with(new Tuple5Tuple3CoGroup());
-
-		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
-		
-		String expected = "1,1,Hallo\n" +
-				"2,2,Hallo Welt\n" +
-				"3,2,Hallo Welt wie gehts?\n" +
-				"3,2,ABC\n" +
-				"5,3,HIJ\n" +
-				"5,3,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithClosureCleaner() throws Exception {
-		/*
-		 * CoGroup with multiple key fields, test working closure cleaner for inner classes
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
-				where(new KeySelector<Tuple5<Integer, Long, Integer, String, Long>,
-						Tuple2<Integer, Long>>() {
-					@Override
-					public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) throws Exception {
-						return new Tuple2<Integer, Long>(t.f0, t.f4);
-					}
-				}).
-				equalTo(new KeySelector<Tuple3<Integer,Long,String>, Tuple2<Integer, Long>>() {
-
-					@Override
-					public Tuple2<Integer, Long> getKey(Tuple3<Integer,Long,String> t) {
-						return new Tuple2<Integer, Long>(t.f0, t.f1);
-					}
-				}).
-				with(new CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>() {
-					@Override
-					public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-					                    Iterable<Tuple3<Integer, Long, String>> second,
-					                    Collector<Tuple3<Integer, Long, String>> out)
-					{
-						List<String> strs = new ArrayList<String>();
-
-						for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
-							strs.add(t.f3);
-						}
-
-						for(Tuple3<Integer, Long, String> t : second) {
-							for(String s : strs) {
-								out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
-							}
-						}
-					}
-				});
-
-		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
-
-		String expected = "1,1,Hallo\n" +
-				"2,2,Hallo Welt\n" +
-				"3,2,Hallo Welt wie gehts?\n" +
-				"3,2,ABC\n" +
-				"5,3,HIJ\n" +
-				"5,3,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithMultipleKeyFieldsWithInnerClassKeyExtractorWithoutClosureCleaner() throws Exception {
-		/*
-		 * CoGroup with multiple key fields, test that disabling closure cleaner leads to an exception when using inner
-		 * classes.
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().disableClosureCleaner();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-		boolean correctExceptionTriggered = false;
-		try {
-			DataSet<Tuple3<Integer, Long, String>> coGrouped = ds1.coGroup(ds2).
-					where(new KeySelector<Tuple5<Integer, Long, Integer, String, Long>,
-							Tuple2<Integer, Long>>() {
-						@Override
-						public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) throws Exception {
-							return new Tuple2<Integer, Long>(t.f0, t.f4);
-						}
-					}).
-					equalTo(new KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>>() {
-
-						@Override
-						public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
-							return new Tuple2<Integer, Long>(t.f0, t.f1);
-						}
-					}).
-					with(new CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>() {
-						@Override
-						public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-						                    Iterable<Tuple3<Integer, Long, String>> second,
-						                    Collector<Tuple3<Integer, Long, String>> out) {
-							List<String> strs = new ArrayList<String>();
-
-							for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
-								strs.add(t.f3);
-							}
-
-							for (Tuple3<Integer, Long, String> t : second) {
-								for (String s : strs) {
-									out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
-								}
-							}
-						}
-					});
-		} catch (InvalidProgramException ex) {
-			correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException);
-		}
-		Assert.assertTrue(correctExceptionTriggered);
-
-	}
-
-	public static class KeySelector7 implements KeySelector<Tuple5<Integer,Long,Integer,String,Long>,
-	Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple5<Integer,Long,Integer,String,Long> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f4);
-		}
-	}
-
-	public static class KeySelector8 implements KeySelector<Tuple3<Integer,Long,String>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple3<Integer,Long,String> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f1);
-		}
-	}
-
-	@Test
-	public void testCoGroupTwoCustomTypeInputsWithExpressionKeys() throws Exception {
-		/*
-		 * CoGroup on two custom type inputs using expression keys
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> coGroupDs = ds.coGroup(ds2).where("myInt").equalTo("myInt").with(new CustomTypeCoGroup());
-
-		List<CustomType> result = coGroupDs.collect();
-		
-		String expected = "1,0,test\n" +
-				"2,6,test\n" +
-				"3,24,test\n" +
-				"4,60,test\n" +
-				"5,120,test\n" +
-				"6,210,test\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCoGroupOnTwoCustomTypeInputsWithExpressionKeyAndFieldSelector() throws
-			Exception {
-		/*
-		 * CoGroup on two custom type inputs using expression keys
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
-				.where("nestedPojo.longNumber").equalTo(6).with(new CoGroup1());
-
-		List<CustomType> result = coGroupDs.collect();
-		
-		String expected = 	"-1,20000,Flink\n" +
-				"-1,10000,Flink\n" +
-				"-1,30000,Flink\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class CoGroup1 implements CoGroupFunction<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<POJO> first,
-				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
-				Collector<CustomType> out) throws Exception {
-			for(POJO p : first) {
-				for(Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
-					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
-					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
-				}
-			}
-		}
-	}
-
-	@Test
-	public void testCoGroupFieldSelectorAndComplicatedKeySelector() throws Exception {
-		/*
-		 * CoGroup field-selector (expression keys) + key selector function
-		 * The key selector is unnecessary complicated (Tuple1) ;)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
-				.where(new KeySelector6()).equalTo(6).with(new CoGroup3());
-
-		List<CustomType> result = coGroupDs.collect();
-		
-		String expected = 	"-1,20000,Flink\n" +
-				"-1,10000,Flink\n" +
-				"-1,30000,Flink\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector6 implements KeySelector<POJO, Tuple1<Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple1<Long> getKey(POJO value)
-		throws Exception {
-			return new Tuple1<Long>(value.nestedPojo.longNumber);
-		}
-	}
-
-	public static class CoGroup3 implements CoGroupFunction<POJO, Tuple7<Integer,
-			String, Integer, Integer, Long, String, Long>, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<POJO> first,
-				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
-				Collector<CustomType> out) throws Exception {
-			for(POJO p : first) {
-				for(Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
-					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
-					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
-				}
-			}
-		}
-	}
-
-	@Test
-	public void testCoGroupFieldSelectorAndKeySelector() throws Exception {
-		/*
-		 * CoGroup field-selector (expression keys) + key selector function
-		 * The key selector is simple here
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds = CollectionDataSets.getSmallPojoDataSet(env);
-		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
-		DataSet<CustomType> coGroupDs = ds.coGroup(ds2)
-				.where(new KeySelector1()).equalTo(6).with(new CoGroup2());
-
-		List<CustomType> result = coGroupDs.collect();
-		
-		String expected = "-1,20000,Flink\n" +
-				"-1,10000,Flink\n" +
-				"-1,30000,Flink\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithAtomicType1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Integer> ds2 = env.fromElements(0, 1, 2);
-
-		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds1.coGroup(ds2).where(0).equalTo("*").with(new CoGroupAtomic1());
-
-		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
-		
-		String expected = "(1,1,Hi)\n" +
-			"(2,2,Hello)";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithAtomicType2() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Integer> ds1 = env.fromElements(0, 1, 2);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> coGroupDs = ds1.coGroup(ds2).where("*").equalTo(0).with(new CoGroupAtomic2());
-		
-		List<Tuple3<Integer, Long, String>> result = coGroupDs.collect();
-		
-		String expected = "(1,1,Hi)\n" +
-			"(2,2,Hello)";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCoGroupWithRangePartitioning() throws Exception {
-		/*
-		 * Test coGroup on tuples with multiple key field positions and same customized distribution
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds1 = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
-
-		env.setParallelism(4);
-		TestDistribution testDis = new TestDistribution();
-		DataSet<Tuple3<Integer, Long, String>> coGrouped =
-				DataSetUtils.partitionByRange(ds1, testDis, 0, 4)
-						.coGroup(DataSetUtils.partitionByRange(ds2, testDis, 0, 1))
-						.where(0, 4)
-						.equalTo(0, 1)
-						.with(new Tuple5Tuple3CoGroup());
-
-		List<Tuple3<Integer, Long, String>> result = coGrouped.collect();
-
-		String expected = "1,1,Hallo\n" +
-				"2,2,Hallo Welt\n" +
-				"3,2,Hallo Welt wie gehts?\n" +
-				"3,2,ABC\n" +
-				"5,3,HIJ\n" +
-				"5,3,IJK\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-
-	// --------------------------------------------------------------------------------------------
-	//  UDF classes
-	// --------------------------------------------------------------------------------------------
-	
-	public static class KeySelector1 implements KeySelector<POJO, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long getKey(POJO value)
-		throws Exception {
-			return value.nestedPojo.longNumber;
-		}
-	}
-
-	public static class CoGroup2 implements CoGroupFunction<POJO, Tuple7<Integer, String,
-			Integer, Integer, Long, String, Long>, CustomType> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<POJO> first,
-				Iterable<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> second,
-				Collector<CustomType> out) throws Exception {
-			for(POJO p : first) {
-				for(Tuple7<Integer, String, Integer, Integer, Long, String, Long> t: second) {
-					Assert.assertTrue(p.nestedPojo.longNumber == t.f6);
-					out.collect(new CustomType(-1, p.nestedPojo.longNumber, "Flink"));
-				}
-			}
-		}
-	}
-
-	public static class Tuple5CoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
-				Collector<Tuple2<Integer, Integer>> out)
-		{
-			int sum = 0;
-			int id = 0;
-			
-			for ( Tuple5<Integer, Long, Integer, String, Long> element : first ) {
-				sum += element.f2;
-				id = element.f0;
-			}
-			
-			for ( Tuple5<Integer, Long, Integer, String, Long> element : second ) {
-				sum += element.f2;
-				id = element.f0;
-			}
-			
-			out.collect(new Tuple2<Integer, Integer>(id, sum));
-		}
-	}
-	
-	public static class CustomTypeCoGroup implements CoGroupFunction<CustomType, CustomType, CustomType> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<CustomType> first, Iterable<CustomType> second, Collector<CustomType> out) {
-			
-			CustomType o = new CustomType(0,0,"test");
-			
-			for ( CustomType element : first ) {
-				o.myInt = element.myInt;
-				o.myLong += element.myLong;
-			}
-			
-			for ( CustomType element : second ) {
-				o.myInt = element.myInt;
-				o.myLong += element.myLong;
-			}
-			
-			out.collect(o);
-		}
-	}
-	
-	public static class MixedCoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, CustomType, Tuple3<Integer, Long, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-				Iterable<CustomType> second,
-				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			
-			long sum = 0;
-			int id = 0;
-			
-			for ( Tuple5<Integer, Long, Integer, String, Long> element : first ) {
-				sum += element.f0;
-				id = element.f2;
-			}
-			
-			for (CustomType element : second) {
-				id = element.myInt;
-				sum += element.myLong;
-			}
-			
-			out.collect(new Tuple3<Integer, Long, String>(id, sum, "test"));
-		}
-		
-	}
-	
-	public static class MixedCoGroup2 implements CoGroupFunction<CustomType, Tuple5<Integer, Long, Integer, String, Long>, CustomType> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<CustomType> first,
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
-				Collector<CustomType> out)
-		{
-			CustomType o = new CustomType(0,0,"test");
-			
-			for (CustomType element : first) {
-				o.myInt = element.myInt;
-				o.myLong += element.myLong;
-			}
-			
-			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
-				o.myInt = element.f2;
-				o.myLong += element.f0;
-			}
-			
-			out.collect(o);
-			
-		}
-		
-	}
-	
-	public static class Tuple3ReturnLeft implements CoGroupFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<Tuple3<Integer, Long, String>> first,
-				Iterable<Tuple3<Integer, Long, String>> second,
-				Collector<Tuple3<Integer, Long, String>> out)
-		{
-			for (Tuple3<Integer, Long, String> element : first) {
-				if(element.f0 < 6) {
-					out.collect(element);
-				}
-			}
-		}
-	}
-	
-	public static class Tuple5ReturnRight implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
-		
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
-				Collector<Tuple5<Integer, Long, Integer, String, Long>> out)
-		{
-			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
-				if(element.f0 < 4) {
-					out.collect(element);
-				}
-			}
-		}
-	}
-	
-	public static class Tuple5CoGroupBC extends RichCoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-		
-		private int broadcast = 42;
-		
-		@Override
-		public void open(Configuration config) {
-			
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			broadcast = sum;
-			
-		}
-
-		@Override
-		public void coGroup(
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-				Iterable<Tuple5<Integer, Long, Integer, String, Long>> second,
-				Collector<Tuple3<Integer, Integer, Integer>> out)
-		{
-			int sum = 0;
-			int id = 0;
-			
-			for (Tuple5<Integer, Long, Integer, String, Long> element : first) {
-				sum += element.f2;
-				id = element.f0;
-			}
-			
-			for (Tuple5<Integer, Long, Integer, String, Long> element : second) {
-				sum += element.f2;
-				id = element.f0;
-			}
-			
-			out.collect(new Tuple3<Integer, Integer, Integer>(id, sum, broadcast));
-		}
-	}
-	
-	public static class Tuple5Tuple3CoGroup implements CoGroupFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-		
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<Tuple5<Integer, Long, Integer, String, Long>> first,
-				Iterable<Tuple3<Integer, Long, String>> second,
-				Collector<Tuple3<Integer, Long, String>> out)
-		{
-			List<String> strs = new ArrayList<String>();
-			
-			for (Tuple5<Integer, Long, Integer, String, Long> t : first) {
-				strs.add(t.f3);
-			}
-			
-			for(Tuple3<Integer, Long, String> t : second) {
-				for(String s : strs) {
-					out.collect(new Tuple3<Integer, Long, String>(t.f0, t.f1, s));
-				}
-			}
-		}
-	}
-
-	public static class CoGroupAtomic1 implements CoGroupFunction<Tuple3<Integer, Long, String>, Integer, Tuple3<Integer, Long, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<Tuple3<Integer, Long, String>> first, Iterable<Integer> second, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			List<Integer> ints = new ArrayList<Integer>();
-
-			for (Integer i : second) {
-				ints.add(i);
-			}
-
-			for (Tuple3<Integer, Long, String> t : first) {
-				for (Integer i : ints) {
-					if (t.f0.equals(i)) {
-						out.collect(t);
-					}
-				}
-			}
-		}
-	}
-
-	public static class CoGroupAtomic2 implements CoGroupFunction<Integer, Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<Integer> first, Iterable<Tuple3<Integer, Long, String>> second, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
-			List<Integer> ints = new ArrayList<Integer>();
-
-			for (Integer i : first) {
-				ints.add(i);
-			}
-
-			for (Tuple3<Integer, Long, String> t : second) {
-				for (Integer i : ints) {
-					if (t.f0.equals(i)) {
-						out.collect(t);
-					}
-				}
-			}
-		}
-	}
-
-	public static class TestDistribution implements DataDistribution {
-		public Object[][] boundaries = new Object[][]{
-				new Object[]{2, 2L},
-				new Object[]{5, 4L},
-				new Object[]{10, 12L},
-				new Object[]{21, 6L}
-		};
-
-		public TestDistribution() {}
-
-		@Override
-		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
-			return boundaries[bucketNum];
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 2;
-		}
-
-		@Override
-		public TypeInformation[] getKeyTypes() {
-			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO};
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-
-		}
-
-		@Override
-		public void read(DataInputView in) throws IOException {
-
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj instanceof TestDistribution; 
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
deleted file mode 100644
index 63d1ec7..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CrossITCase.java
+++ /dev/null
@@ -1,456 +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.javaApiOperators;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.common.functions.RichCrossFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class CrossITCase extends MultipleProgramsTestBase {
-
-	public CrossITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testCorretnessOfCrossOnTwoTupleInputs() throws Exception {
-		/*
-		 * check correctness of cross on two tuple inputs
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<Integer, String>> crossDs = ds.cross(ds2).with(new Tuple5Cross());
-
-		List<Tuple2<Integer, String>> result = crossDs.collect();
-
-		String expected = "0,HalloHallo\n" +
-				"1,HalloHallo Welt\n" +
-				"2,HalloHallo Welt wie\n" +
-				"1,Hallo WeltHallo\n" +
-				"2,Hallo WeltHallo Welt\n" +
-				"3,Hallo WeltHallo Welt wie\n" +
-				"2,Hallo Welt wieHallo\n" +
-				"3,Hallo Welt wieHallo Welt\n" +
-				"4,Hallo Welt wieHallo Welt wie\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossIfUDFReturnsLeftInputObject() throws Exception {
-		/*
-		 * check correctness of cross if UDF returns left input object
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> crossDs = ds.cross(ds2).with(new Tuple3ReturnLeft());
-
-		List<Tuple3<Integer, Long, String>> result = crossDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"1,1,Hi\n" +
-				"1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"2,2,Hello\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"3,2,Hello world\n" +
-				"3,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossIfUDFReturnsRightInputObject() throws Exception {
-		/*
-		 * check correctness of cross if UDF returns right input object
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> crossDs = ds.cross(ds2).with(new Tuple5ReturnRight());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = crossDs
-				.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"1,1,0,Hallo,1\n" +
-				"1,1,0,Hallo,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"2,3,2,Hallo Welt wie,1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossWithBroadcastSet() throws Exception {
-		/*
-		 * check correctness of cross with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple3<Integer, Integer, Integer>> crossDs = ds.cross(ds2).with(new Tuple5CrossBC()).withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<Integer, Integer, Integer>> result = crossDs.collect();
-
-		String expected = "2,0,55\n" +
-				"3,0,55\n" +
-				"3,0,55\n" +
-				"3,0,55\n" +
-				"4,1,55\n" +
-				"4,2,55\n" +
-				"3,0,55\n" +
-				"4,2,55\n" +
-				"4,4,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossWithHuge() throws Exception {
-		/*
-		 * check correctness of crossWithHuge (only correctness of result -> should be the same as with normal cross)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<Integer, String>> crossDs = ds.crossWithHuge(ds2).with(new Tuple5Cross());
-
-		List<Tuple2<Integer, String>> result = crossDs.collect();
-
-		String expected = "0,HalloHallo\n" +
-				"1,HalloHallo Welt\n" +
-				"2,HalloHallo Welt wie\n" +
-				"1,Hallo WeltHallo\n" +
-				"2,Hallo WeltHallo Welt\n" +
-				"3,Hallo WeltHallo Welt wie\n" +
-				"2,Hallo Welt wieHallo\n" +
-				"3,Hallo Welt wieHallo Welt\n" +
-				"4,Hallo Welt wieHallo Welt wie\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossWithTiny() throws Exception {
-		/*
-		 * check correctness of crossWithTiny (only correctness of result -> should be the same as with normal cross)
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<Integer, String>> crossDs = ds.crossWithTiny(ds2).with(new Tuple5Cross());
-
-		List<Tuple2<Integer, String>> result = crossDs.collect();
-
-		String expected = "0,HalloHallo\n" +
-				"1,HalloHallo Welt\n" +
-				"2,HalloHallo Welt wie\n" +
-				"1,Hallo WeltHallo\n" +
-				"2,Hallo WeltHallo Welt\n" +
-				"3,Hallo WeltHallo Welt wie\n" +
-				"2,Hallo Welt wieHallo\n" +
-				"3,Hallo Welt wieHallo Welt\n" +
-				"4,Hallo Welt wieHallo Welt wie\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testProjectCrossOnATupleInput1() throws Exception{
-		/*
-		 * project cross on a tuple input 1
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple6<String, Long, String, Integer, Long, Long>> crossDs = ds.cross(ds2)
-				.projectFirst(2, 1)
-				.projectSecond(3)
-				.projectFirst(0)
-				.projectSecond(4,1);
-
-		List<Tuple6<String, Long, String, Integer, Long, Long>> result = crossDs.collect();
-
-		String expected = "Hi,1,Hallo,1,1,1\n" +
-				"Hi,1,Hallo Welt,1,2,2\n" +
-				"Hi,1,Hallo Welt wie,1,1,3\n" +
-				"Hello,2,Hallo,2,1,1\n" +
-				"Hello,2,Hallo Welt,2,2,2\n" +
-				"Hello,2,Hallo Welt wie,2,1,3\n" +
-				"Hello world,2,Hallo,3,1,1\n" +
-				"Hello world,2,Hallo Welt,3,2,2\n" +
-				"Hello world,2,Hallo Welt wie,3,1,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testProjectCrossOnATupleInput2() throws Exception {
-		/*
-		 * project cross on a tuple input 2
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple6<String, String, Long, Long, Long,Integer>> crossDs = ds.cross(ds2)
-				.projectSecond(3)
-				.projectFirst(2, 1)
-				.projectSecond(4,1)
-				.projectFirst(0);
-
-		List<Tuple6<String, String, Long, Long, Long, Integer>> result = crossDs.collect();
-
-		String expected = "Hallo,Hi,1,1,1,1\n" +
-				"Hallo Welt,Hi,1,2,2,1\n" +
-				"Hallo Welt wie,Hi,1,1,3,1\n" +
-				"Hallo,Hello,2,1,1,2\n" +
-				"Hallo Welt,Hello,2,2,2,2\n" +
-				"Hallo Welt wie,Hello,2,1,3,2\n" +
-				"Hallo,Hello world,2,1,1,3\n" +
-				"Hallo Welt,Hello world,2,2,2,3\n" +
-				"Hallo Welt wie,Hello world,2,1,3,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDefaultCross() throws Exception {
-		/*
-		 * check correctness of default cross
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> crossDs = ds.cross(ds2);
-
-		List<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> result = crossDs.collect();
-
-		String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n"
-				+
-				"(1,1,Hi),(1,1,0,Hallo,1)\n" +
-				"(1,1,Hi),(2,3,2,Hallo Welt wie,1)\n" +
-				"(2,2,Hello),(2,2,1,Hallo Welt,2)\n" +
-				"(2,2,Hello),(1,1,0,Hallo,1)\n" +
-				"(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" +
-				"(3,2,Hello world),(2,2,1,Hallo Welt,2)\n" +
-				"(3,2,Hello world),(1,1,0,Hallo,1)\n" +
-				"(3,2,Hello world),(2,3,2,Hallo Welt wie,1)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossOnTwoCustomTypeInputs() throws Exception {
-		/*
-		 * check correctness of cross on two custom type inputs
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<CustomType> crossDs = ds.cross(ds2).with(new CustomTypeCross());
-
-		List<CustomType> result = crossDs.collect();
-
-		String expected = "1,0,HiHi\n"
-				+ "2,1,HiHello\n"
-				+ "2,2,HiHello world\n"
-				+ "2,1,HelloHi\n"
-				+ "4,2,HelloHello\n"
-				+ "4,3,HelloHello world\n"
-				+ "2,2,Hello worldHi\n"
-				+ "4,3,Hello worldHello\n"
-				+ "4,4,Hello worldHello world";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfCrossATupleInputAndACustomTypeInput() throws Exception {
-		/*
-		 * check correctness of cross a tuple input and a custom type input
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> crossDs = ds.cross(ds2).with(new MixedCross());
-
-		List<Tuple3<Integer, Long, String>> result = crossDs.collect();
-
-		String expected = "2,0,HalloHi\n" +
-				"3,0,HalloHello\n" +
-				"3,0,HalloHello world\n" +
-				"3,0,Hallo WeltHi\n" +
-				"4,1,Hallo WeltHello\n" +
-				"4,2,Hallo WeltHello world\n" +
-				"3,0,Hallo Welt wieHi\n" +
-				"4,2,Hallo Welt wieHello\n" +
-				"4,4,Hallo Welt wieHello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Tuple5Cross implements CrossFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-
-		@Override
-		public Tuple2<Integer, String> cross(
-				Tuple5<Integer, Long, Integer, String, Long> first,
-				Tuple5<Integer, Long, Integer, String, Long> second)
-				throws Exception {
-
-				return new Tuple2<Integer, String>(first.f2+second.f2, first.f3+second.f3);
-		}
-
-	}
-
-	public static class CustomTypeCross implements CrossFunction<CustomType, CustomType, CustomType> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public CustomType cross(CustomType first, CustomType second)
-				throws Exception {
-
-			return new CustomType(first.myInt * second.myInt, first.myLong + second.myLong, first.myString + second.myString);
-		}
-
-	}
-
-	public static class MixedCross implements CrossFunction<Tuple5<Integer, Long, Integer, String, Long>, CustomType, Tuple3<Integer, Long, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple3<Integer, Long, String> cross(
-				Tuple5<Integer, Long, Integer, String, Long> first,
-				CustomType second) throws Exception {
-
-			return new Tuple3<Integer, Long, String>(first.f0 + second.myInt, first.f2 * second.myLong, first.f3 + second.myString);
-		}
-
-	}
-
-
-	public static class Tuple3ReturnLeft implements CrossFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple3<Integer, Long, String> cross(
-				Tuple3<Integer, Long, String> first,
-				Tuple5<Integer, Long, Integer, String, Long> second) throws Exception {
-
-			return first;
-		}
-	}
-
-	public static class Tuple5ReturnRight implements CrossFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple5<Integer, Long, Integer, String, Long> cross(
-				Tuple3<Integer, Long, String> first,
-				Tuple5<Integer, Long, Integer, String, Long> second)
-				throws Exception {
-
-			return second;
-		}
-
-
-	}
-
-	public static class Tuple5CrossBC extends RichCrossFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private int broadcast = 42;
-
-		@Override
-		public void open(Configuration config) {
-
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			broadcast = sum;
-
-		}
-
-		@Override
-		public Tuple3<Integer, Integer, Integer> cross(
-				Tuple5<Integer, Long, Integer, String, Long> first,
-				Tuple5<Integer, Long, Integer, String, Long> second)
-				throws Exception {
-
-			return new Tuple3<Integer, Integer, Integer>(first.f0 + second.f0, first.f2 * second.f2, broadcast);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
deleted file mode 100644
index 24a09ba..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
+++ /dev/null
@@ -1,359 +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.javaApiOperators;
-
-import org.apache.flink.api.common.distributions.DataDistribution;
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.test.util.TestEnvironment;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.fail;
-
-@SuppressWarnings("serial")
-public class CustomDistributionITCase extends TestLogger {
-
-	// ------------------------------------------------------------------------
-	//  The mini cluster that is shared across tests
-	// ------------------------------------------------------------------------
-
-	private static LocalFlinkMiniCluster cluster;
-
-	@BeforeClass
-	public static void setup() throws Exception {
-		cluster = TestBaseUtils.startCluster(1, 8, false, false, true);
-	}
-
-	@AfterClass
-	public static void teardown() throws Exception {
-		TestBaseUtils.stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
-	}
-
-	@Before
-	public void prepare() {
-		TestEnvironment clusterEnv = new TestEnvironment(cluster, 1, false);
-		clusterEnv.setAsContext();
-	}
-
-	@After
-	public void cleanup() {
-		TestEnvironment.unsetAsContext();
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Test the record partitioned rightly with one field according to the customized data distribution
-	 */
-	@Test
-	public void testPartitionWithDistribution1() throws Exception {
-		final TestDataDist1 dist = new TestDataDist1();
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(dist.getParallelism());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Boolean> result = DataSetUtils
-			.partitionByRange(input, dist, 0)
-			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Long, String>, Boolean>() {
-
-				@Override
-				public void mapPartition(Iterable<Tuple3<Integer, Long, String>> values, Collector<Boolean> out) throws Exception {
-					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
-
-					for (Tuple3<Integer, Long, String> s : values) {
-						boolean correctlyPartitioned = true;
-						if (pIdx == 0) {
-							Integer[] upper = dist.boundaries[0];
-							if (s.f0.compareTo(upper[0]) > 0) {
-								correctlyPartitioned = false;
-							}
-						}
-						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							Integer[] upper = dist.boundaries[pIdx];
-							if (s.f0.compareTo(upper[0]) > 0 || (s.f0.compareTo(lower[0]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-						else {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							if ((s.f0.compareTo(lower[0]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-
-						if (!correctlyPartitioned) {
-							fail("Record was not correctly partitioned: " + s.toString());
-						}
-					}
-				}
-			}
-			);
-
-		result.output(new DiscardingOutputFormat<Boolean>());
-		env.execute();
-	}
-
-	/**
-	 * Test the record partitioned rightly with two fields according to the customized data distribution
-	 */
-	@Test
-	public void testRangeWithDistribution2() throws Exception {
-		final TestDataDist2 dist = new TestDataDist2();
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(dist.getParallelism());
-
-		DataSet<Tuple3<Integer, Integer, String>> input = env.fromElements(
-						new Tuple3<>(1, 5, "Hi"),
-						new Tuple3<>(1, 6, "Hi"),
-						new Tuple3<>(1, 7, "Hi"),
-						new Tuple3<>(1, 11, "Hello"),
-						new Tuple3<>(2, 3, "World"),
-						new Tuple3<>(2, 4, "World"),
-						new Tuple3<>(2, 5, "World"),
-						new Tuple3<>(2, 13, "Hello World"),
-						new Tuple3<>(3, 8, "Say"),
-						new Tuple3<>(4, 0, "Why"),
-						new Tuple3<>(4, 2, "Java"),
-						new Tuple3<>(4, 11, "Say Hello"),
-						new Tuple3<>(5, 1, "Hi Java!"),
-						new Tuple3<>(5, 2, "Hi Java?"),
-						new Tuple3<>(5, 3, "Hi Java again")
-			);
-
-		DataSet<Boolean> result = DataSetUtils
-			.partitionByRange(input, dist, 0, 1)
-			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Integer, String>, Boolean>() {
-
-				@Override
-				public void mapPartition(Iterable<Tuple3<Integer, Integer, String>> values, Collector<Boolean> out) throws Exception {
-					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
-					boolean correctlyPartitioned = true;
-
-					for (Tuple3<Integer, Integer, String> s : values) {
-
-						if (pIdx == 0) {
-							Integer[] upper = dist.boundaries[0];
-							if (s.f0.compareTo(upper[0]) > 0 ||
-								(s.f0.compareTo(upper[0]) == 0 && s.f1.compareTo(upper[1]) > 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							Integer[] upper = dist.boundaries[pIdx];
-
-							if (s.f0.compareTo(upper[0]) > 0 ||
-								(s.f0.compareTo(upper[0]) == 0 && s.f1.compareTo(upper[1]) > 0) ||
-								(s.f0.compareTo(lower[0]) < 0) ||
-								(s.f0.compareTo(lower[0]) == 0 && s.f1.compareTo(lower[1]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-						else {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							if ((s.f0.compareTo(lower[0]) < 0) ||
-								(s.f0.compareTo(lower[0]) == 0 && s.f1.compareTo(lower[1]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-
-						if (!correctlyPartitioned) {
-							fail("Record was not correctly partitioned: " + s.toString());
-						}
-					}
-				}
-			}
-			);
-
-		result.output(new DiscardingOutputFormat<Boolean>());
-		env.execute();
-	}
-
-	/*
-	 * Test the number of partition keys less than the number of distribution fields
-	 */
-	@Test
-	public void testPartitionKeyLessDistribution() throws Exception {
-		final TestDataDist2 dist = new TestDataDist2();
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(dist.getParallelism());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-
-		DataSet<Boolean> result = DataSetUtils
-			.partitionByRange(input, dist, 0)
-			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Long, String>, Boolean>() {
-
-				@Override
-				public void mapPartition(Iterable<Tuple3<Integer, Long, String>> values, Collector<Boolean> out) throws Exception {
-					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
-
-					for (Tuple3<Integer, Long, String> s : values) {
-						boolean correctlyPartitioned = true;
-						if (pIdx == 0) {
-							Integer[] upper = dist.boundaries[0];
-							if (s.f0.compareTo(upper[0]) > 0) {
-								correctlyPartitioned = false;
-							}
-						}
-						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							Integer[] upper = dist.boundaries[pIdx];
-							if (s.f0.compareTo(upper[0]) > 0 || (s.f0.compareTo(lower[0]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-						else {
-							Integer[] lower = dist.boundaries[pIdx - 1];
-							if ((s.f0.compareTo(lower[0]) <= 0)) {
-								correctlyPartitioned = false;
-							}
-						}
-
-						if (!correctlyPartitioned) {
-							fail("Record was not correctly partitioned: " + s.toString());
-						}
-					}
-				}
-			}
-			);
-
-		result.output(new DiscardingOutputFormat<Boolean>());
-		env.execute();
-	}
-
-	/*
-	 * Test the number of partition keys larger than the number of distribution fields
-	 */
-	@Test(expected = IllegalArgumentException.class)
-	public void testPartitionMoreThanDistribution() throws Exception {
-		final TestDataDist2 dist = new TestDataDist2();
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		DataSetUtils.partitionByRange(input, dist, 0, 1, 2);
-	}
-	
-	/**
-	 * The class is used to do the tests of range partition with one key.
-	 */
-	public static class TestDataDist1 implements DataDistribution {
-
-		public Integer boundaries[][] = new Integer[][]{
-			new Integer[]{4},
-			new Integer[]{9},
-			new Integer[]{13},
-			new Integer[]{18}
-		};
-
-		public TestDataDist1() {}
-
-		public int getParallelism() {
-			return boundaries.length;
-		}
-
-		@Override
-		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
-			return boundaries[bucketNum];
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 1;
-		}
-
-		@Override
-		public TypeInformation[] getKeyTypes() {
-			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO};
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {}
-
-		@Override
-		public void read(DataInputView in) throws IOException {}
-	}
-
-	/**
-	 * The class is used to do the tests of range partition with two keys.
-	 */
-	public static class TestDataDist2 implements DataDistribution {
-
-		public Integer boundaries[][] = new Integer[][]{
-			new Integer[]{1, 6},
-			new Integer[]{2, 4},
-			new Integer[]{3, 9},
-			new Integer[]{4, 1},
-			new Integer[]{5, 2}
-		};
-
-		public TestDataDist2() {}
-
-		public int getParallelism() {
-			return boundaries.length;
-		}
-		
-		@Override
-		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
-			return boundaries[bucketNum];
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 2;
-		}
-
-		@Override
-		public TypeInformation[] getKeyTypes() {
-			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO};
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {}
-
-		@Override
-		public void read(DataInputView in) throws IOException {}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
deleted file mode 100644
index fb62ccd..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSinkITCase.java
+++ /dev/null
@@ -1,355 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.BufferedReader;
-import java.util.Random;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for data sinks
- */
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class DataSinkITCase extends MultipleProgramsTestBase {
-
-	public DataSinkITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	private String resultPath;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception{
-		resultPath = tempFolder.newFile().toURI().toString();
-	}
-
-	@Test
-	public void testIntSortingParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> ds = CollectionDataSets.getIntegerDataSet(env);
-		ds.writeAsText(resultPath).sortLocalOutput("*", Order.DESCENDING).setParallelism(1);
-
-		env.execute();
-
-		String expected = "5\n5\n5\n5\n5\n4\n4\n4\n4\n3\n3\n3\n2\n2\n1\n";
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-
-	}
-
-	@Test
-	public void testStringSortingParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		ds.writeAsText(resultPath).sortLocalOutput("*", Order.ASCENDING).setParallelism(1);
-
-		env.execute();
-
-		String expected = "Hello\n" +
-				"Hello world\n" +
-				"Hello world, how are you?\n" +
-				"Hi\n" +
-				"I am fine.\n" +
-				"LOL\n" +
-				"Luke Skywalker\n" +
-				"Random comment\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testTupleSortingSingleAscParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.ASCENDING).setParallelism(1);
-
-		env.execute();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n" +
-				"6,3,Luke Skywalker\n" +
-				"7,4,Comment#1\n" +
-				"8,4,Comment#2\n" +
-				"9,4,Comment#3\n" +
-				"10,4,Comment#4\n" +
-				"11,5,Comment#5\n" +
-				"12,5,Comment#6\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" +
-				"15,5,Comment#9\n" +
-				"16,6,Comment#10\n" +
-				"17,6,Comment#11\n" +
-				"18,6,Comment#12\n" +
-				"19,6,Comment#13\n" +
-				"20,6,Comment#14\n" +
-				"21,6,Comment#15\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testTupleSortingSingleDescParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.DESCENDING).setParallelism(1);
-
-		env.execute();
-
-		String expected = "21,6,Comment#15\n" +
-				"20,6,Comment#14\n" +
-				"19,6,Comment#13\n" +
-				"18,6,Comment#12\n" +
-				"17,6,Comment#11\n" +
-				"16,6,Comment#10\n" +
-				"15,5,Comment#9\n" +
-				"14,5,Comment#8\n" +
-				"13,5,Comment#7\n" +
-				"12,5,Comment#6\n" +
-				"11,5,Comment#5\n" +
-				"10,4,Comment#4\n" +
-				"9,4,Comment#3\n" +
-				"8,4,Comment#2\n" +
-				"7,4,Comment#1\n" +
-				"6,3,Luke Skywalker\n" +
-				"5,3,I am fine.\n" +
-				"4,3,Hello world, how are you?\n" +
-				"3,2,Hello world\n" +
-				"2,2,Hello\n" +
-				"1,1,Hi\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testTupleSortingDualParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		ds.writeAsCsv(resultPath)
-			.sortLocalOutput(1, Order.DESCENDING).sortLocalOutput(0, Order.ASCENDING)
-			.setParallelism(1);
-
-		env.execute();
-
-		String expected = "16,6,Comment#10\n" +
-				"17,6,Comment#11\n" +
-				"18,6,Comment#12\n" +
-				"19,6,Comment#13\n" +
-				"20,6,Comment#14\n" +
-				"21,6,Comment#15\n" +
-				"11,5,Comment#5\n" +
-				"12,5,Comment#6\n" +
-				"13,5,Comment#7\n" +
-				"14,5,Comment#8\n" +
-				"15,5,Comment#9\n" +
-				"7,4,Comment#1\n" +
-				"8,4,Comment#2\n" +
-				"9,4,Comment#3\n" +
-				"10,4,Comment#4\n" +
-				"4,3,Hello world, how are you?\n" +
-				"5,3,I am fine.\n" +
-				"6,3,Luke Skywalker\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n" +
-				"1,1,Hi\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testTupleSortingNestedParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> ds =
-				CollectionDataSets.getGroupSortedNestedTupleDataSet2(env);
-		ds.writeAsText(resultPath)
-			.sortLocalOutput("f0.f1", Order.ASCENDING)
-			.sortLocalOutput("f1", Order.DESCENDING)
-			.setParallelism(1);
-
-		env.execute();
-
-		String expected =
-				"((2,1),a,3)\n" +
-				"((2,2),b,4)\n" +
-				"((1,2),a,1)\n" +
-				"((3,3),c,5)\n" +
-				"((1,3),a,2)\n" +
-				"((3,6),c,6)\n" +
-				"((4,9),c,7)\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testTupleSortingNestedParallelism1_2() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> ds =
-				CollectionDataSets.getGroupSortedNestedTupleDataSet2(env);
-		ds.writeAsText(resultPath)
-			.sortLocalOutput(1, Order.ASCENDING)
-			.sortLocalOutput(2, Order.DESCENDING)
-			.setParallelism(1);
-
-		env.execute();
-
-		String expected =
-				"((2,1),a,3)\n" +
-				"((1,3),a,2)\n" +
-				"((1,2),a,1)\n" +
-				"((2,2),b,4)\n" +
-				"((4,9),c,7)\n" +
-				"((3,6),c,6)\n" +
-				"((3,3),c,5)\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testPojoSortingSingleParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
-		ds.writeAsText(resultPath).sortLocalOutput("number", Order.ASCENDING).setParallelism(1);
-
-		env.execute();
-
-		String expected = "1 First (10,100,1000,One) 10100\n" +
-				"2 First_ (10,105,1000,One) 10200\n" +
-				"3 First (11,102,3000,One) 10200\n" +
-				"4 First_ (11,106,1000,One) 10300\n" +
-				"5 First (11,102,2000,One) 10100\n" +
-				"6 Second_ (20,200,2000,Two) 10100\n" +
-				"7 Third (31,301,2000,Three) 10200\n" +
-				"8 Third_ (30,300,1000,Three) 10100\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testPojoSortingDualParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
-		ds.writeAsText(resultPath)
-			.sortLocalOutput("str", Order.ASCENDING)
-			.sortLocalOutput("number", Order.DESCENDING)
-			.setParallelism(1);
-
-		env.execute();
-
-		String expected =
-				"5 First (11,102,2000,One) 10100\n" +
-				"3 First (11,102,3000,One) 10200\n" +
-				"1 First (10,100,1000,One) 10100\n" +
-				"4 First_ (11,106,1000,One) 10300\n" +
-				"2 First_ (10,105,1000,One) 10200\n" +
-				"6 Second_ (20,200,2000,Two) 10100\n" +
-				"7 Third (31,301,2000,Three) 10200\n" +
-				"8 Third_ (30,300,1000,Three) 10100\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-
-	}
-
-	@Test
-	public void testPojoSortingNestedParallelism1() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
-		ds.writeAsText(resultPath)
-			.sortLocalOutput("nestedTupleWithCustom.f0", Order.ASCENDING)
-			.sortLocalOutput("nestedTupleWithCustom.f1.myInt", Order.DESCENDING)
-			.sortLocalOutput("nestedPojo.longNumber", Order.ASCENDING)
-			.setParallelism(1);
-
-		env.execute();
-
-		String expected =
-				"2 First_ (10,105,1000,One) 10200\n" +
-				"1 First (10,100,1000,One) 10100\n" +
-				"4 First_ (11,106,1000,One) 10300\n" +
-				"5 First (11,102,2000,One) 10100\n" +
-				"3 First (11,102,3000,One) 10200\n" +
-				"6 Second_ (20,200,2000,Two) 10100\n" +
-				"8 Third_ (30,300,1000,Three) 10100\n" +
-				"7 Third (31,301,2000,Three) 10200\n";
-
-		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
-	}
-
-	@Test
-	public void testSortingParallelism4() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Long> ds = env.generateSequence(0, 1000);
-		// randomize
-		ds.map(new MapFunction<Long, Long>() {
-
-			Random rand = new Random(1234L);
-			@Override
-			public Long map(Long value) throws Exception {
-				return rand.nextLong();
-			}
-		}).writeAsText(resultPath)
-			.sortLocalOutput("*", Order.ASCENDING)
-			.setParallelism(4);
-
-		env.execute();
-
-		BufferedReader[] resReaders = getResultReader(resultPath);
-		for (BufferedReader br : resReaders) {
-			long cmp = Long.MIN_VALUE;
-			while(br.ready()) {
-				long cur = Long.parseLong(br.readLine());
-				assertTrue("Invalid order of sorted output", cmp <= cur);
-				cmp = cur;
-			}
-			br.close();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.java
deleted file mode 100644
index aa40754..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DataSourceITCase.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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.junit.Assert;
-
-/**
- * Tests for the DataSource
- */
-public class DataSourceITCase extends JavaProgramTestBase {
-
-	private String inputPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		inputPath = createTempFile("input", "ab\n"
-				+ "cd\n"
-				+ "ef\n");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		/*
-		 * Test passing a configuration object to an input format
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		Configuration ifConf = new Configuration();
-		ifConf.setString("prepend", "test");
-
-		DataSet<String> ds = env.createInput(new TestInputFormat(new Path(inputPath))).withParameters(ifConf);
-		List<String> result = ds.collect();
-
-		String expectedResult = "ab\n"
-				+ "cd\n"
-				+ "ef\n";
-
-		compareResultAsText(result, expectedResult);
-	}
-
-	private static class TestInputFormat extends TextInputFormat {
-		private static final long serialVersionUID = 1L;
-
-		public TestInputFormat(Path filePath) {
-			super(filePath);
-		}
-
-		@Override
-		public void configure(Configuration parameters) {
-			super.configure(parameters);
-
-			Assert.assertNotNull(parameters.getString("prepend", null));
-			Assert.assertEquals("test", parameters.getString("prepend", null));
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
deleted file mode 100644
index 6e30fac..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/DistinctITCase.java
+++ /dev/null
@@ -1,318 +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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class DistinctITCase extends MultipleProgramsTestBase {
-
-	public DistinctITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelector() throws Exception {
-		/*
-		 * check correctness of distinct on tuples with key field selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct(0, 1, 2);
-
-		List<Tuple3<Integer, Long, String>> result = distinctDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllFieldsSelected() throws Exception{
-		/*
-		 * check correctness of distinct on tuples with key field selector with not all fields selected
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple1<Integer>> distinctDs = ds.union(ds).distinct(0).project(0);
-
-		List<Tuple1<Integer>> result = distinctDs.collect();
-
-		String expected = "1\n" +
-				"2\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnTuplesWithKeyExtractorFunction() throws Exception {
-		/*
-		 * check correctness of distinct on tuples with key extractor function
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple1<Integer>> reduceDs = ds.union(ds).distinct(new KeySelector1()).project(0);
-
-		List<Tuple1<Integer>> result = reduceDs.collect();
-
-		String expected = "1\n" + "2\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector1 implements KeySelector<Tuple5<Integer, Long,  Integer, String, Long>, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(Tuple5<Integer, Long,  Integer, String, Long> in) {
-			return in.f0;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnCustomTypeWithTypeExtractor() throws Exception {
-		/*
-		 * check correctness of distinct on custom type with type extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<Tuple1<Integer>> reduceDs = ds
-				.distinct(new KeySelector3())
-				.map(new Mapper3());
-
-		List<Tuple1<Integer>> result = reduceDs.collect();
-
-		String expected = "1\n" +
-				"2\n" +
-				"3\n" +
-				"4\n" +
-				"5\n" +
-				"6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class Mapper3 extends RichMapFunction<CustomType, Tuple1<Integer>> {
-		@Override
-		public Tuple1<Integer> map(CustomType value) throws Exception {
-			return new Tuple1<Integer>(value.myInt);
-		}
-	}
-
-	public static class KeySelector3 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnTuples() throws Exception{
-		/*
-		 * check correctness of distinct on tuples
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct();
-
-		List<Tuple3<Integer, Long, String>> result = distinctDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"2,2,Hello\n" +
-				"3,2,Hello world\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor() throws Exception{
-		/*
-		 * check correctness of distinct on custom type with tuple-returning type extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> reduceDs = ds
-				.distinct(new KeySelector2())
-				.project(0,4);
-
-		List<Tuple2<Integer, Long>> result = reduceDs.collect();
-
-		String expected = "1,1\n" +
-				"2,1\n" +
-				"2,2\n" +
-				"3,2\n" +
-				"3,3\n" +
-				"4,1\n" +
-				"4,2\n" +
-				"5,1\n" +
-				"5,2\n" +
-				"5,3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<Integer,Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f4);
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnTuplesWithFieldExpressions() throws Exception {
-		/*
-		 * check correctness of distinct on tuples with field expressions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
-		DataSet<Tuple1<Integer>> reduceDs = ds.union(ds).distinct("f0").project(0);
-
-		List<Tuple1<Integer>> result = reduceDs.collect();
-
-		String expected = "1\n" +
-				"2\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnPojos() throws Exception {
-		/*
-		 * check correctness of distinct on Pojos
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
-		DataSet<Integer> reduceDs = ds.distinct("nestedPojo.longNumber").map(new Mapper2());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "10000\n20000\n30000\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper2 implements MapFunction<CollectionDataSets.POJO, Integer> {
-		@Override
-		public Integer map(POJO value) throws Exception {
-			return (int) value.nestedPojo.longNumber;
-		}
-	}
-
-	@Test
-	public void testDistinctOnFullPojo() throws Exception {
-		/*
-		 * distinct on full Pojo
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
-		DataSet<Integer> reduceDs = ds.distinct().map(new Mapper1());
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "10000\n20000\n30000\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper1 implements MapFunction<CollectionDataSets.POJO, Integer> {
-		@Override
-		public Integer map(POJO value) throws Exception {
-			return (int) value.nestedPojo.longNumber;
-		}
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnAtomic() throws Exception {
-		/*
-		 * check correctness of distinct on Integers
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<Integer> ds = CollectionDataSets.getIntegerDataSet(env);
-		DataSet<Integer> reduceDs = ds.distinct();
-
-		List<Integer> result = reduceDs.collect();
-
-		String expected = "1\n2\n3\n4\n5";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testCorrectnessOfDistinctOnAtomicWithSelectAllChar() throws Exception {
-		/*
-		 * check correctness of distinct on Strings, using Keys.ExpressionKeys.SELECT_ALL_CHAR
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
-		DataSet<String> reduceDs = ds.union(ds).distinct("*");
-
-		List<String> result = reduceDs.collect();
-
-		String expected = "I am fine.\n" +
-				"Luke Skywalker\n" +
-				"LOL\n" +
-				"Hello world, how are you?\n" +
-				"Hi\n" +
-				"Hello world\n" +
-				"Hello\n" +
-				"Random comment\n";
-
-		compareResultAsText(result, expected);
-	}
-}


[04/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java
index e134c7a..e1a5d87 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/ConnectedComponentsCoGroupTest.java
@@ -41,11 +41,12 @@ import org.apache.flink.optimizer.plan.SourcePlanNode;
 import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 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.optimizer.util.CompilerTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -54,93 +55,92 @@ import org.junit.Test;
  */
 @SuppressWarnings("serial")
 public class ConnectedComponentsCoGroupTest extends CompilerTestBase {
-	
+
 	private static final String VERTEX_SOURCE = "Vertices";
-	
+
 	private static final String ITERATION_NAME = "Connected Components Iteration";
-	
+
 	private static final String EDGES_SOURCE = "Edges";
 	private static final String JOIN_NEIGHBORS_MATCH = "Join Candidate Id With Neighbor";
 	private static final String MIN_ID_AND_UPDATE = "Min Id and Update";
-	
+
 	private static final String SINK = "Result";
-	
+
 	private static final boolean PRINT_PLAN = false;
-	
+
 	private final FieldList set0 = new FieldList(0);
-	
-	
+
 	@Test
 	public void testWorksetConnectedComponents() {
 		Plan plan = getConnectedComponentsCoGroupPlan();
 		plan.setExecutionConfig(new ExecutionConfig());
 		OptimizedPlan optPlan = compileNoStats(plan);
 		OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan);
-		
+
 		if (PRINT_PLAN) {
 			PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();
 			String json = dumper.getOptimizerPlanAsJSON(optPlan);
 			System.out.println(json);
 		}
-		
+
 		SourcePlanNode vertexSource = or.getNode(VERTEX_SOURCE);
 		SourcePlanNode edgesSource = or.getNode(EDGES_SOURCE);
 		SinkPlanNode sink = or.getNode(SINK);
 		WorksetIterationPlanNode iter = or.getNode(ITERATION_NAME);
-		
+
 		DualInputPlanNode neighborsJoin = or.getNode(JOIN_NEIGHBORS_MATCH);
 		DualInputPlanNode cogroup = or.getNode(MIN_ID_AND_UPDATE);
-		
+
 		// --------------------------------------------------------------------
 		// Plan validation:
 		//
 		// We expect the plan to go with a sort-merge join, because the CoGroup
 		// sorts and the join in the successive iteration can re-exploit the sorting.
 		// --------------------------------------------------------------------
-		
+
 		// test all drivers
 		Assert.assertEquals(DriverStrategy.NONE, sink.getDriverStrategy());
 		Assert.assertEquals(DriverStrategy.NONE, vertexSource.getDriverStrategy());
 		Assert.assertEquals(DriverStrategy.NONE, edgesSource.getDriverStrategy());
-		
+
 		Assert.assertEquals(DriverStrategy.INNER_MERGE, neighborsJoin.getDriverStrategy());
 		Assert.assertEquals(set0, neighborsJoin.getKeysForInput1());
 		Assert.assertEquals(set0, neighborsJoin.getKeysForInput2());
-		
+
 		Assert.assertEquals(DriverStrategy.CO_GROUP, cogroup.getDriverStrategy());
 		Assert.assertEquals(set0, cogroup.getKeysForInput1());
 		Assert.assertEquals(set0, cogroup.getKeysForInput2());
-		
+
 		// test all the shipping strategies
 		Assert.assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
 		Assert.assertEquals(ShipStrategyType.PARTITION_HASH, iter.getInitialSolutionSetInput().getShipStrategy());
 		Assert.assertEquals(set0, iter.getInitialSolutionSetInput().getShipStrategyKeys());
 		Assert.assertEquals(ShipStrategyType.PARTITION_HASH, iter.getInitialWorksetInput().getShipStrategy());
 		Assert.assertEquals(set0, iter.getInitialWorksetInput().getShipStrategyKeys());
-		
+
 		Assert.assertEquals(ShipStrategyType.FORWARD, neighborsJoin.getInput1().getShipStrategy()); // workset
 		Assert.assertEquals(ShipStrategyType.PARTITION_HASH, neighborsJoin.getInput2().getShipStrategy()); // edges
 		Assert.assertEquals(set0, neighborsJoin.getInput2().getShipStrategyKeys());
 		Assert.assertTrue(neighborsJoin.getInput2().getTempMode().isCached());
-		
+
 		Assert.assertEquals(ShipStrategyType.PARTITION_HASH, cogroup.getInput1().getShipStrategy()); // min id
 		Assert.assertEquals(ShipStrategyType.FORWARD, cogroup.getInput2().getShipStrategy()); // solution set
-		
+
 		// test all the local strategies
 		Assert.assertEquals(LocalStrategy.NONE, sink.getInput().getLocalStrategy());
 		Assert.assertEquals(LocalStrategy.NONE, iter.getInitialSolutionSetInput().getLocalStrategy());
-		
+
 		// the sort for the neighbor join in the first iteration is pushed out of the loop
 		Assert.assertEquals(LocalStrategy.SORT, iter.getInitialWorksetInput().getLocalStrategy());
 		Assert.assertEquals(LocalStrategy.NONE, neighborsJoin.getInput1().getLocalStrategy()); // workset
 		Assert.assertEquals(LocalStrategy.SORT, neighborsJoin.getInput2().getLocalStrategy()); // edges
-		
+
 		Assert.assertEquals(LocalStrategy.SORT, cogroup.getInput1().getLocalStrategy());
 		Assert.assertEquals(LocalStrategy.NONE, cogroup.getInput2().getLocalStrategy()); // solution set
-		
+
 		// check the caches
 		Assert.assertTrue(TempMode.CACHED == neighborsJoin.getInput2().getTempMode());
-		
+
 		JobGraphGenerator jgg = new JobGraphGenerator();
 		jgg.compileJobGraph(optPlan);
 	}
@@ -150,17 +150,17 @@ public class ConnectedComponentsCoGroupTest extends CompilerTestBase {
 		PreviewPlanEnvironment env = new PreviewPlanEnvironment();
 		env.setAsContext();
 		try {
-			ConnectedComponentsWithCoGroup(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE, "100"});
+			connectedComponentsWithCoGroup(new String[]{DEFAULT_PARALLELISM_STRING, IN_FILE, IN_FILE, OUT_FILE, "100"});
 		} catch (ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
-			Assert.fail("ConnectedComponentsWithCoGroup failed with an exception");
+			Assert.fail("connectedComponentsWithCoGroup failed with an exception");
 		}
 		return env.getPlan();
 	}
 
-	public static void ConnectedComponentsWithCoGroup(String[] args) throws Exception {
+	public static void connectedComponentsWithCoGroup(String[] args) throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		env.setParallelism(Integer.parseInt(args[0]));
 
@@ -170,8 +170,8 @@ public class ConnectedComponentsCoGroupTest extends CompilerTestBase {
 
 		DataSet<Tuple2<Long, Long>> verticesWithId = initialVertices.flatMap(new DummyMapFunction());
 
-		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration
-				= verticesWithId.iterateDelta(verticesWithId, Integer.parseInt(args[4]), 0).name(ITERATION_NAME);
+		DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration =
+				verticesWithId.iterateDelta(verticesWithId, Integer.parseInt(args[4]), 0).name(ITERATION_NAME);
 
 		DataSet<Tuple2<Long, Long>> joinWithNeighbors = iteration.getWorkset().join(edges)
 				.where(0).equalTo(0)
@@ -186,14 +186,14 @@ public class ConnectedComponentsCoGroupTest extends CompilerTestBase {
 		env.execute();
 	}
 
-	public static class DummyMapFunction implements FlatMapFunction<Tuple1<Long>, Tuple2<Long, Long>> {
+	private static class DummyMapFunction implements FlatMapFunction<Tuple1<Long>, Tuple2<Long, Long>> {
 		@Override
 		public void flatMap(Tuple1<Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
 			// won't be executed
 		}
 	}
 
-	public static class DummyJoinFunction implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class DummyJoinFunction implements FlatJoinFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		@Override
 		public void join(Tuple2<Long, Long> first, Tuple2<Long, Long> second, Collector<Tuple2<Long, Long>> out) throws Exception {
 			// won't be executed
@@ -202,7 +202,7 @@ public class ConnectedComponentsCoGroupTest extends CompilerTestBase {
 
 	@ForwardedFieldsFirst("f0->f0")
 	@ForwardedFieldsSecond("f0->f0")
-	public static class DummyCoGroupFunction implements CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class DummyCoGroupFunction implements CoGroupFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		@Override
 		public void coGroup(Iterable<Tuple2<Long, Long>> first, Iterable<Tuple2<Long, Long>> second, Collector<Tuple2<Long, Long>> out) throws Exception {
 			// won't be executed

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
index f17b28a..f4f35e7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/MultipleJoinsWithSolutionSetCompilerTest.java
@@ -18,68 +18,72 @@
 
 package org.apache.flink.test.optimizer.iterations;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichJoinFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.DeltaIteration;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.runtime.operators.DriverStrategy;
-import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
-import org.apache.flink.optimizer.util.CompilerTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.operators.DeltaIteration;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.optimizer.plan.DualInputPlanNode;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plan.SolutionSetPlanNode;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.CompilerTestBase;
+import org.apache.flink.runtime.operators.DriverStrategy;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
+/**
+ * Test multiple joins with the solution set.
+ */
 @SuppressWarnings("serial")
 public class MultipleJoinsWithSolutionSetCompilerTest extends CompilerTestBase {
-		
+
 	private static final String JOIN_1 = "join1";
 	private static final String JOIN_2 = "join2";
-		
+
 	@Test
 	public void testMultiSolutionSetJoinPlan() {
 		try {
-			
+
 			ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
+
 			@SuppressWarnings("unchecked")
 			DataSet<Tuple2<Long, Double>> inputData = env.fromElements(new Tuple2<Long, Double>(1L, 1.0));
 			DataSet<Tuple2<Long, Double>> result = constructPlan(inputData, 10);
-			
+
 			// add two sinks, to test the case of branching after an iteration
 			result.output(new DiscardingOutputFormat<Tuple2<Long, Double>>());
 			result.output(new DiscardingOutputFormat<Tuple2<Long, Double>>());
-		
+
 			Plan p = env.createProgramPlan();
-			
+
 			OptimizedPlan optPlan = compileNoStats(p);
-			
+
 			OptimizerPlanNodeResolver or = getOptimizerPlanNodeResolver(optPlan);
-			
+
 			DualInputPlanNode join1 = or.getNode(JOIN_1);
 			DualInputPlanNode join2 = or.getNode(JOIN_2);
-			
+
 			assertEquals(DriverStrategy.HYBRIDHASH_BUILD_FIRST, join1.getDriverStrategy());
 			assertEquals(DriverStrategy.HYBRIDHASH_BUILD_SECOND, join2.getDriverStrategy());
-			
+
 			assertEquals(ShipStrategyType.PARTITION_HASH, join1.getInput2().getShipStrategy());
 			assertEquals(ShipStrategyType.PARTITION_HASH, join2.getInput1().getShipStrategy());
-			
+
 			assertEquals(SolutionSetPlanNode.class, join1.getInput1().getSource().getClass());
 			assertEquals(SolutionSetPlanNode.class, join2.getInput2().getSource().getClass());
-			
+
 			new JobGraphGenerator().compileJobGraph(optPlan);
 		}
 		catch (Exception e) {
@@ -88,42 +92,40 @@ public class MultipleJoinsWithSolutionSetCompilerTest extends CompilerTestBase {
 			fail("Test erroneous: " + e.getMessage());
 		}
 	}
-	
-	
-	
+
 	public static DataSet<Tuple2<Long, Double>> constructPlan(DataSet<Tuple2<Long, Double>> initialData, int numIterations) {
 
 		DeltaIteration<Tuple2<Long, Double>, Tuple2<Long, Double>> iteration = initialData.iterateDelta(initialData, numIterations, 0);
-		
+
 		DataSet<Tuple2<Long, Double>> delta = iteration.getSolutionSet()
 				.join(iteration.getWorkset().flatMap(new Duplicator())).where(0).equalTo(0).with(new SummingJoin()).name(JOIN_1)
 				.groupBy(0).aggregate(Aggregations.MIN, 1).map(new Expander())
 				.join(iteration.getSolutionSet()).where(0).equalTo(0).with(new SummingJoinProject()).name(JOIN_2);
-		
+
 		DataSet<Tuple2<Long, Double>> changes = delta.groupBy(0).aggregate(Aggregations.SUM, 1);
-		
+
 		DataSet<Tuple2<Long, Double>> result = iteration.closeWith(delta, changes);
-		
+
 		return result;
 	}
-	
-	public static final class SummingJoin extends RichJoinFunction<Tuple2<Long, Double>, Tuple2<Long, Double>, Tuple2<Long, Double>> {
+
+	private static final class SummingJoin extends RichJoinFunction<Tuple2<Long, Double>, Tuple2<Long, Double>, Tuple2<Long, Double>> {
 
 		@Override
 		public Tuple2<Long, Double> join(Tuple2<Long, Double> first, Tuple2<Long, Double> second) {
 			return new Tuple2<Long, Double>(first.f0, first.f1 + second.f1);
 		}
 	}
-	
-	public static final class SummingJoinProject extends RichJoinFunction<Tuple3<Long, Double, Double>, Tuple2<Long, Double>, Tuple2<Long, Double>> {
+
+	private static final class SummingJoinProject extends RichJoinFunction<Tuple3<Long, Double, Double>, Tuple2<Long, Double>, Tuple2<Long, Double>> {
 
 		@Override
 		public Tuple2<Long, Double> join(Tuple3<Long, Double, Double> first, Tuple2<Long, Double> second) {
 			return new Tuple2<Long, Double>(first.f0, first.f1 + first.f2 + second.f1);
 		}
 	}
-	
-	public static final class Duplicator extends RichFlatMapFunction<Tuple2<Long, Double>, Tuple2<Long, Double>> {
+
+	private static final class Duplicator extends RichFlatMapFunction<Tuple2<Long, Double>, Tuple2<Long, Double>> {
 
 		@Override
 		public void flatMap(Tuple2<Long, Double> value, Collector<Tuple2<Long, Double>> out) {
@@ -131,8 +133,8 @@ public class MultipleJoinsWithSolutionSetCompilerTest extends CompilerTestBase {
 			out.collect(value);
 		}
 	}
-	
-	public static final class Expander extends RichMapFunction<Tuple2<Long, Double>, Tuple3<Long, Double, Double>> {
+
+	private static final class Expander extends RichMapFunction<Tuple2<Long, Double>, Tuple3<Long, Double, Double>> {
 
 		@Override
 		public Tuple3<Long, Double, Double> map(Tuple2<Long, Double> value) {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java
index f916151..d7ce29d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/iterations/PageRankCompilerTest.java
@@ -16,61 +16,64 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.optimizer.iterations;
 
-import static org.apache.flink.api.java.aggregation.Aggregations.SUM;
-import static org.junit.Assert.fail;
-
 import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.IterativeDataSet;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.plan.BulkIterationPlanNode;
-import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plan.SinkPlanNode;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.examples.java.graph.PageRank.BuildOutgoingEdgeList;
 import org.apache.flink.examples.java.graph.PageRank.Dampener;
 import org.apache.flink.examples.java.graph.PageRank.EpsilonFilter;
 import org.apache.flink.examples.java.graph.PageRank.JoinVertexWithEdgesMatch;
 import org.apache.flink.examples.java.graph.PageRank.RankAssigner;
+import org.apache.flink.optimizer.Optimizer;
+import org.apache.flink.optimizer.plan.BulkIterationPlanNode;
+import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.util.CompilerTestBase;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
-import org.apache.flink.optimizer.util.CompilerTestBase;
+
 import org.junit.Assert;
 import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.IterativeDataSet;
 
+import static org.apache.flink.api.java.aggregation.Aggregations.SUM;
+import static org.junit.Assert.fail;
+
+/**
+ * Test compilation of PageRank implementation.
+ */
 public class PageRankCompilerTest extends CompilerTestBase{
-	
+
 	@Test
 	public void testPageRank() {
 		try {
 			final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-			
+
 			// get input data
-			DataSet<Long> pagesInput = env.fromElements(1l);
+			DataSet<Long> pagesInput = env.fromElements(1L);
 			@SuppressWarnings("unchecked")
-			DataSet<Tuple2<Long, Long>> linksInput =env.fromElements(new Tuple2<Long, Long>(1l, 2l));
-			
+			DataSet<Tuple2<Long, Long>> linksInput = env.fromElements(new Tuple2<Long, Long>(1L, 2L));
+
 			// assign initial rank to pages
 			DataSet<Tuple2<Long, Double>> pagesWithRanks = pagesInput.
 					map(new RankAssigner((1.0d / 10)));
-			
+
 			// build adjacency list from link input
-			DataSet<Tuple2<Long, Long[]>> adjacencyListInput = 
+			DataSet<Tuple2<Long, Long[]>> adjacencyListInput =
 					linksInput.groupBy(0).reduceGroup(new BuildOutgoingEdgeList());
-			
+
 			// set iterative data set
 			IterativeDataSet<Tuple2<Long, Double>> iteration = pagesWithRanks.iterate(10);
-			
+
 			Configuration cfg = new Configuration();
 			cfg.setString(Optimizer.HINT_LOCAL_STRATEGY, Optimizer.HINT_LOCAL_STRATEGY_HASH_BUILD_SECOND);
-			
+
 			DataSet<Tuple2<Long, Double>> newRanks = iteration
 					// join pages with outgoing edges and distribute rank
 					.join(adjacencyListInput).where(0).equalTo(0).withParameters(cfg)
@@ -79,26 +82,26 @@ public class PageRankCompilerTest extends CompilerTestBase{
 					.groupBy(0).aggregate(SUM, 1)
 					// apply dampening factor
 					.map(new Dampener(0.85, 10));
-			
+
 			DataSet<Tuple2<Long, Double>> finalPageRanks = iteration.closeWith(
-					newRanks, 
+					newRanks,
 					newRanks.join(iteration).where(0).equalTo(0)
 					// termination condition
 					.filter(new EpsilonFilter()));
-	
+
 			finalPageRanks.output(new DiscardingOutputFormat<Tuple2<Long, Double>>());
-	
+
 			// get the plan and compile it
 			Plan p = env.createProgramPlan();
 			OptimizedPlan op = compileNoStats(p);
-			
+
 			SinkPlanNode sinkPlanNode = (SinkPlanNode) op.getDataSinks().iterator().next();
 			BulkIterationPlanNode iterPlanNode = (BulkIterationPlanNode) sinkPlanNode.getInput().getSource();
-			
+
 			// check that the partitioning is pushed out of the first loop
 			Assert.assertEquals(ShipStrategyType.PARTITION_HASH, iterPlanNode.getInput().getShipStrategy());
 			Assert.assertEquals(LocalStrategy.NONE, iterPlanNode.getInput().getLocalStrategy());
-			
+
 			BulkPartialSolutionPlanNode partSolPlanNode = iterPlanNode.getPartialSolutionPlanNode();
 			Assert.assertEquals(ShipStrategyType.FORWARD, partSolPlanNode.getOutgoingChannels().get(0).getShipStrategy());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java
index 6932fd2..d066d8e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java
@@ -18,30 +18,31 @@
 
 package org.apache.flink.test.optimizer.jsonplan;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonParser;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.client.program.OptimizerPlanEnvironment;
 import org.apache.flink.client.program.PreviewPlanEnvironment;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.examples.java.clustering.KMeans;
 import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.examples.java.graph.PageRank;
 import org.apache.flink.examples.java.relational.TPCHQuery3;
 import org.apache.flink.examples.java.relational.WebLogAnalysis;
 import org.apache.flink.examples.java.wordcount.WordCount;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.util.CompilerTestBase;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
 import org.junit.Assert;
 import org.junit.Test;
 
-/*
+/**
  * The tests in this class simply invokes the JSON dump code for the optimized plan.
  */
 public class DumpCompiledPlanTest extends CompilerTestBase {
-	
+
 	@Test
 	public void dumpWordCount() {
 		// prepare the test environment
@@ -51,7 +52,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 			WordCount.main(new String[] {
 					"--input", IN_FILE,
 					"--output", OUT_FILE});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -59,7 +60,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpTPCH3() {
 		// prepare the test environment
@@ -71,7 +72,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 					"--customer", IN_FILE,
 					"--orders", OUT_FILE,
 					"--output", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -79,7 +80,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpIterativeKMeans() {
 		// prepare the test environment
@@ -91,7 +92,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 				"--centroids ", IN_FILE,
 				"--output ", OUT_FILE,
 				"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -99,7 +100,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpWebLogAnalysis() {
 		// prepare the test environment
@@ -111,7 +112,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 					"--ranks", IN_FILE,
 					"--visits", OUT_FILE,
 					"--output", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -131,7 +132,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 					"--edges", IN_FILE,
 					"--output", OUT_FILE,
 					"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -139,7 +140,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpPageRank() {
 		// prepare the test environment
@@ -152,7 +153,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 					"--output", OUT_FILE,
 					"--numPages", "10",
 					"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -160,7 +161,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	private void dump(Plan p) {
 		p.setExecutionConfig(new ExecutionConfig());
 		try {
@@ -168,7 +169,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase {
 			PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();
 			String json = dumper.getOptimizerPlanAsJSON(op);
 			JsonParser parser = new JsonFactory().createJsonParser(json);
-			while (parser.nextToken() != null);
+			while (parser.nextToken() != null) {}
 		} catch (JsonParseException e) {
 			e.printStackTrace();
 			Assert.fail("JSON Generator produced malformatted output: " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
index 0c14d82..3174801 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/JsonJobGraphGenerationTest.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.test.optimizer.jsonplan;
 
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -37,10 +31,14 @@ import org.apache.flink.examples.java.wordcount.WordCount;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
-
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -51,8 +49,14 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+/**
+ * Test job graph generation in JSON format.
+ */
 public class JsonJobGraphGenerationTest {
 
 	private PrintStream out;
@@ -67,11 +71,11 @@ public class JsonJobGraphGenerationTest {
 			@Override
 			public void write(int b) {}
 		};
-		
+
 		System.setOut(new PrintStream(discards));
 		System.setErr(new PrintStream(discards));
 	}
-	
+
 	@After
 	public void restoreStreams() {
 		if (out != null) {
@@ -81,8 +85,7 @@ public class JsonJobGraphGenerationTest {
 			System.setOut(err);
 		}
 	}
-	
-	
+
 	@Test
 	public void testWordCountPlan() {
 		try {
@@ -95,13 +98,13 @@ public class JsonJobGraphGenerationTest {
 				WordCount.main(new String[0]);
 			}
 			catch (AbortError ignored) {}
-			
+
 			// with arguments
 			try {
 				final int parallelism = 17;
 				JsonValidator validator = new GenericValidator(parallelism, 3);
 				TestingExecutionEnvironment.setAsNext(validator, parallelism);
-				
+
 				String tmpDir = ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH;
 				WordCount.main(new String[] {
 						"--input", tmpDir,
@@ -186,7 +189,7 @@ public class JsonJobGraphGenerationTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testConnectedComponents() {
 		try {
@@ -214,7 +217,7 @@ public class JsonJobGraphGenerationTest {
 						"--iterations", "100");
 			}
 			catch (AbortError ignored) {}
-			
+
 		}
 		catch (Exception e) {
 			restoreStreams();
@@ -224,12 +227,12 @@ public class JsonJobGraphGenerationTest {
 	}
 
 	// ------------------------------------------------------------------------
-	
-	private static interface JsonValidator {
-		
+
+	private interface JsonValidator {
+
 		void validateJson(String json) throws Exception;
 	}
-	
+
 	private static class GenericValidator implements JsonValidator {
 
 		private final int expectedParallelism;
@@ -243,7 +246,7 @@ public class JsonJobGraphGenerationTest {
 		@Override
 		public void validateJson(String json) throws Exception {
 			final Map<String, JsonNode> idToNode = new HashMap<>();
-			
+
 			// validate the produced JSON
 			ObjectMapper m = new ObjectMapper();
 			JsonNode rootNode = m.readTree(json);
@@ -251,7 +254,7 @@ public class JsonJobGraphGenerationTest {
 			JsonNode idField = rootNode.get("jid");
 			JsonNode nameField = rootNode.get("name");
 			JsonNode arrayField = rootNode.get("nodes");
-			
+
 			assertNotNull(idField);
 			assertNotNull(nameField);
 			assertNotNull(arrayField);
@@ -260,14 +263,15 @@ public class JsonJobGraphGenerationTest {
 			assertTrue(arrayField.isArray());
 
 			ArrayNode array = (ArrayNode) arrayField;
-			for (Iterator<JsonNode> iter = array.elements(); iter.hasNext(); ) {
+			Iterator<JsonNode> iter = array.elements();
+			while (iter.hasNext()) {
 				JsonNode vertex = iter.next();
 
 				JsonNode vertexIdField = vertex.get("id");
 				JsonNode parallelismField = vertex.get("parallelism");
 				JsonNode contentsFields = vertex.get("description");
 				JsonNode operatorField = vertex.get("operator");
-				
+
 				assertNotNull(vertexIdField);
 				assertTrue(vertexIdField.isTextual());
 				assertNotNull(parallelismField);
@@ -276,30 +280,31 @@ public class JsonJobGraphGenerationTest {
 				assertTrue(contentsFields.isTextual());
 				assertNotNull(operatorField);
 				assertTrue(operatorField.isTextual());
-				
+
 				if (contentsFields.asText().startsWith("Sync")) {
 					assertEquals(1, parallelismField.asInt());
 				}
 				else {
 					assertEquals(expectedParallelism, parallelismField.asInt());
 				}
-				
+
 				idToNode.put(vertexIdField.asText(), vertex);
 			}
-			
+
 			assertEquals(numNodes, idToNode.size());
-			
+
 			// check that all inputs are contained
 			for (JsonNode node : idToNode.values()) {
 				JsonNode inputsField = node.get("inputs");
 				if (inputsField != null) {
-					for (Iterator<JsonNode> inputsIter = inputsField.elements(); inputsIter.hasNext(); ) {
+					Iterator<JsonNode> inputsIter = inputsField.elements();
+					while (inputsIter.hasNext()) {
 						JsonNode inputNode = inputsIter.next();
 						JsonNode inputIdField = inputNode.get("id");
-						
+
 						assertNotNull(inputIdField);
 						assertTrue(inputIdField.isTextual());
-						
+
 						String inputIdString = inputIdField.asText();
 						assertTrue(idToNode.containsKey(inputIdString));
 					}
@@ -309,15 +314,15 @@ public class JsonJobGraphGenerationTest {
 	}
 
 	// ------------------------------------------------------------------------
-	
+
 	private static class AbortError extends Error {
 		private static final long serialVersionUID = 152179957828703919L;
 	}
-	
+
 	// ------------------------------------------------------------------------
-	
+
 	private static class TestingExecutionEnvironment extends ExecutionEnvironment {
-		
+
 		private final JsonValidator validator;
 
 		private TestingExecutionEnvironment(JsonValidator validator) {
@@ -331,7 +336,7 @@ public class JsonJobGraphGenerationTest {
 		@Override
 		public JobExecutionResult execute(String jobName) throws Exception {
 			Plan plan = createProgramPlan(jobName);
-			
+
 			Optimizer pc = new Optimizer(new Configuration());
 			OptimizedPlan op = pc.compile(plan);
 
@@ -339,13 +344,13 @@ public class JsonJobGraphGenerationTest {
 			JobGraph jobGraph = jgg.compileJobGraph(op);
 
 			String jsonPlan = JsonPlanGenerator.generatePlan(jobGraph);
-			
+
 			// first check that the JSON is valid
 			JsonParser parser = new JsonFactory().createJsonParser(jsonPlan);
-			while (parser.nextToken() != null);
-			
+			while (parser.nextToken() != null) {}
+
 			validator.validateJson(jsonPlan);
-			
+
 			throw new AbortError();
 		}
 
@@ -353,7 +358,7 @@ public class JsonJobGraphGenerationTest {
 		public String getExecutionPlan() throws Exception {
 			throw new UnsupportedOperationException();
 		}
-		
+
 		public static void setAsNext(final JsonValidator validator, final int defaultParallelism) {
 			initializeContextEnvironment(new ExecutionEnvironmentFactory() {
 				@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java
index 14f6656..a487eb0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.optimizer.jsonplan;
 
-import java.util.List;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParseException;
-import com.fasterxml.jackson.core.JsonParser;
 import org.apache.flink.api.common.Plan;
 import org.apache.flink.client.program.OptimizerPlanEnvironment;
 import org.apache.flink.client.program.PreviewPlanEnvironment;
@@ -35,14 +30,20 @@ import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.dag.DataSinkNode;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.util.CompilerTestBase;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
 import org.junit.Assert;
 import org.junit.Test;
 
-/*
+import java.util.List;
+
+/**
  * The tests in this class simply invokes the JSON dump code for the original plan.
  */
 public class PreviewPlanDumpTest extends CompilerTestBase {
-	
+
 	@Test
 	public void dumpWordCount() {
 		// prepare the test environment
@@ -52,7 +53,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 			WordCount.main(new String[] {
 					"--input", IN_FILE,
 					"--output", OUT_FILE});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -60,7 +61,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpTPCH3() {
 		// prepare the test environment
@@ -72,7 +73,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 					"--customer", IN_FILE,
 					"--orders", OUT_FILE,
 					"--output", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -80,7 +81,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpIterativeKMeans() {
 		// prepare the test environment
@@ -92,7 +93,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 				"--centroids ", IN_FILE,
 				"--output ", OUT_FILE,
 				"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -100,7 +101,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpWebLogAnalysis() {
 		// prepare the test environment
@@ -112,7 +113,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 					"--ranks", IN_FILE,
 					"--visits", OUT_FILE,
 					"--output", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -132,7 +133,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 					"--edges", IN_FILE,
 					"--output", OUT_FILE,
 					"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -140,7 +141,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	@Test
 	public void dumpPageRank() {
 		// prepare the test environment
@@ -154,7 +155,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 					"--output", OUT_FILE,
 					"--numPages", "10",
 					"--iterations", "123"});
-		} catch(OptimizerPlanEnvironment.ProgramAbortException pae) {
+		} catch (OptimizerPlanEnvironment.ProgramAbortException pae) {
 			// all good.
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -162,14 +163,14 @@ public class PreviewPlanDumpTest extends CompilerTestBase {
 		}
 		dump(env.getPlan());
 	}
-	
+
 	private void dump(Plan p) {
 		try {
 			List<DataSinkNode> sinks = Optimizer.createPreOptimizedPlan(p);
 			PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();
 			String json = dumper.getPactPlanAsJSON(sinks);
 			try (JsonParser parser = new JsonFactory().createParser(json)) {
-				while (parser.nextToken() != null) ;
+				while (parser.nextToken() != null) {}
 			}
 		} catch (JsonParseException e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
index f012a47..21e3b4c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/AbstractQueryableStateITCase.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.query;
 
-import akka.actor.ActorSystem;
-import akka.dispatch.Futures;
-import akka.dispatch.OnSuccess;
-import akka.dispatch.Recover;
-import akka.pattern.Patterns;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
@@ -60,15 +55,16 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorSystem;
+import akka.dispatch.Futures;
+import akka.dispatch.OnSuccess;
+import akka.dispatch.Recover;
+import akka.pattern.Patterns;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-import scala.reflect.ClassTag$;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -78,6 +74,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLongArray;
 
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+import scala.reflect.ClassTag$;
+
 import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.JobStatusIs;
 import static org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobStatus;
 import static org.junit.Assert.assertEquals;
@@ -92,7 +94,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(100, TimeUnit.SECONDS);
 	private static final FiniteDuration QUERY_RETRY_DELAY = new FiniteDuration(100, TimeUnit.MILLISECONDS);
 
-	private static ActorSystem TEST_ACTOR_SYSTEM;
+	private static ActorSystem testActorSystem;
 
 	private static final int NUM_TMS = 2;
 	private static final int NUM_SLOTS_PER_TM = 4;
@@ -123,7 +125,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			cluster = new TestingCluster(config, false);
 			cluster.start(true);
 
-			TEST_ACTOR_SYSTEM = AkkaUtils.createDefaultActorSystem();
+			testActorSystem = AkkaUtils.createDefaultActorSystem();
 		} catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
@@ -139,8 +141,8 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 			fail(e.getMessage());
 		}
 
-		if (TEST_ACTOR_SYSTEM != null) {
-			TEST_ACTOR_SYSTEM.shutdown();
+		if (testActorSystem != null) {
+			testActorSystem.shutdown();
 		}
 	}
 
@@ -163,7 +165,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	 * number of keys is in fixed in range 0...numKeys). The records are keyed and
 	 * a reducing queryable state instance is created, which sums up the records.
 	 *
-	 * After submitting the job in detached mode, the QueryableStateCLient is used
+	 * <p>After submitting the job in detached mode, the QueryableStateCLient is used
 	 * to query the counts of each key in rounds until all keys have non-zero counts.
 	 */
 	@Test
@@ -255,14 +257,14 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							counts.set(key, result.f1);
 							assertEquals("Key mismatch", key, result.f0.intValue());
 						}
-					}, TEST_ACTOR_SYSTEM.dispatcher());
+					}, testActorSystem.dispatcher());
 
 					futures.add(result);
 				}
 
 				Future<Iterable<Tuple2<Integer, Long>>> futureSequence = Futures.sequence(
 						futures,
-						TEST_ACTOR_SYSTEM.dispatcher());
+						testActorSystem.dispatcher());
 
 				Await.ready(futureSequence, deadline.timeLeft());
 			}
@@ -652,7 +654,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 				new ValueStateDescriptor<>(
 					"any",
 					source.getType(),
-					Tuple2.of(0, 1337l));
+					Tuple2.of(0, 1337L));
 
 			// only expose key "1"
 			QueryableStateStream<Integer, Tuple2<Integer, Long>>
@@ -710,7 +712,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 	 * queried. The tests succeeds after each subtask index is queried with
 	 * value numElements (the latest element updated the state).
 	 *
-	 * This is the same as the simple value state test, but uses the API shortcut.
+	 * <p>This is the same as the simple value state test, but uses the API shortcut.
 	 */
 	@Test
 	public void testValueStateShortcut() throws Exception {
@@ -968,8 +970,8 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							// fail this test.
 							return Patterns.after(
 									retryDelay,
-									TEST_ACTOR_SYSTEM.scheduler(),
-									TEST_ACTOR_SYSTEM.dispatcher(),
+									testActorSystem.scheduler(),
+									testActorSystem.dispatcher(),
 									new Callable<Future<V>>() {
 										@Override
 										public Future<V> call() throws Exception {
@@ -986,7 +988,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 									});
 						}
 					}
-				}, TEST_ACTOR_SYSTEM.dispatcher());
+				}, testActorSystem.dispatcher());
 
 	}
 
@@ -1015,8 +1017,8 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 							// fail this test.
 							return Patterns.after(
 									retryDelay,
-									TEST_ACTOR_SYSTEM.scheduler(),
-									TEST_ACTOR_SYSTEM.dispatcher(),
+									testActorSystem.scheduler(),
+									testActorSystem.dispatcher(),
 									new Callable<Future<V>>() {
 										@Override
 										public Future<V> call() throws Exception {
@@ -1033,7 +1035,7 @@ public abstract class AbstractQueryableStateITCase extends TestLogger {
 									});
 						}
 					}
-				}, TEST_ACTOR_SYSTEM.dispatcher());
+				}, testActorSystem.dispatcher());
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java b/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
index 0f99afb..0c628e4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/KVStateRequestSerializerRocksDBTest.java
@@ -27,18 +27,16 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.contrib.streaming.state.PredefinedOptions;
 import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
-import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializerTest;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.internal.InternalListState;
-
 import org.apache.flink.runtime.state.internal.InternalMapState;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
-
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 
@@ -48,7 +46,8 @@ import static org.mockito.Mockito.mock;
 
 /**
  * Additional tests for the serialization and deserialization of {@link
- * KvStateRequestSerializer} with a RocksDB state back-end.
+ * org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer}
+ * with a RocksDB state back-end.
  */
 public final class KVStateRequestSerializerRocksDBTest {
 
@@ -62,7 +61,7 @@ public final class KVStateRequestSerializerRocksDBTest {
 	 *
 	 * @param <K> key type
 	 */
-	final static class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {
+	static final class RocksDBKeyedStateBackend2<K> extends RocksDBKeyedStateBackend<K> {
 
 		RocksDBKeyedStateBackend2(
 				final String operatorIdentifier,
@@ -127,7 +126,7 @@ public final class KVStateRequestSerializerRocksDBTest {
 
 		KvStateRequestSerializerTest.testListSerialization(key, listState);
 	}
-	
+
 	/**
 	 * Tests map serialization and deserialization match.
 	 *
@@ -158,10 +157,10 @@ public final class KVStateRequestSerializerRocksDBTest {
 		longHeapKeyedStateBackend.restore(null);
 		longHeapKeyedStateBackend.setCurrentKey(key);
 
-		final InternalMapState<VoidNamespace, Long, String> mapState = (InternalMapState<VoidNamespace, Long, String>) 
+		final InternalMapState<VoidNamespace, Long, String> mapState = (InternalMapState<VoidNamespace, Long, String>)
 				longHeapKeyedStateBackend.getPartitionedState(
 						VoidNamespace.INSTANCE,
-						VoidNamespaceSerializer.INSTANCE, 
+						VoidNamespaceSerializer.INSTANCE,
 						new MapStateDescriptor<>("test", LongSerializer.INSTANCE, StringSerializer.INSTANCE));
 
 		KvStateRequestSerializerTest.testMapSerialization(key, mapState);

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseFsBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseFsBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseFsBackend.java
index 6c72bca..b91d277 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseFsBackend.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseFsBackend.java
@@ -20,6 +20,7 @@ package org.apache.flink.test.query;
 
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseRocksDBBackend.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseRocksDBBackend.java b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseRocksDBBackend.java
index 4799da4..9547c5a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseRocksDBBackend.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCaseRocksDBBackend.java
@@ -20,6 +20,7 @@ package org.apache.flink.test.query;
 
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
 import org.apache.flink.runtime.state.AbstractStateBackend;
+
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
index 6882b46..27bd11d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.configuration.AkkaOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -42,17 +36,15 @@ import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.util.NetUtils;
 import org.apache.flink.util.TestLogger;
 
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import org.apache.commons.io.FileUtils;
 import org.junit.Test;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -61,6 +53,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
 import static org.junit.Assert.assertFalse;
@@ -70,7 +68,7 @@ import static org.junit.Assert.fail;
  * Abstract base for tests verifying the behavior of the recovery in the
  * case when a TaskManager fails (process is killed) in the middle of a job execution.
  *
- * The test works with multiple task managers processes by spawning JVMs.
+ * <p>The test works with multiple task managers processes by spawning JVMs.
  * Initially, it starts a JobManager in process and two TaskManagers JVMs with
  * 2 task slots each.
  * It submits a program with parallelism 4 and waits until all tasks are brought up.
@@ -81,8 +79,6 @@ import static org.junit.Assert.fail;
  */
 public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends TestLogger {
 
-	protected final Logger LOG = LoggerFactory.getLogger(getClass());
-
 	protected static final String READY_MARKER_FILE_PREFIX = "ready_";
 	protected static final String PROCEED_MARKER_FILE = "proceed";
 	protected static final String FINISH_MARKER_FILE_PREFIX = "finish_";
@@ -293,10 +289,8 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 	 */
 	public abstract void testTaskManagerFailure(int jobManagerPort, File coordinateDir) throws Exception;
 
-
 	protected void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelayMillis)
-			throws Exception
-	{
+			throws Exception {
 		final long pollInterval = 10_000_000; // 10 ms = 10,000,000 nanos
 		final long deadline = System.nanoTime() + maxDelayMillis * 1_000_000;
 
@@ -360,7 +354,6 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test
 		long now = System.currentTimeMillis();
 		final long deadline = now + timeout;
 
-
 		while (now < deadline) {
 			boolean allFound = true;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
index 0b49814..eb596d4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -26,24 +25,26 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Test;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.fail;
 
+/**
+ * Test program with very fast failure rate.
+ */
 @SuppressWarnings("serial")
 public class FastFailuresITCase extends TestLogger {
 
 	static final AtomicInteger FAILURES_SO_FAR = new AtomicInteger();
 	static final int NUM_FAILURES = 200;
-	
+
 	@Test
 	public void testThis() {
 		final int parallelism = 4;
@@ -51,17 +52,17 @@ public class FastFailuresITCase extends TestLogger {
 		Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
-		
+
 		LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 
 		TestStreamEnvironment env = new TestStreamEnvironment(cluster, parallelism);
-		
+
 		env.getConfig().disableSysoutLogging();
 		env.setParallelism(parallelism);
 		env.enableCheckpointing(1000);
 		env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(210, 0));
-		
+
 		DataStream<Tuple2<Integer, Integer>> input = env.addSource(new RichSourceFunction<Tuple2<Integer, Integer>>() {
 
 			@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
index f70cc1e..51868af 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.PoisonPill;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.test.TestingServer;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
@@ -70,6 +65,12 @@ import org.apache.flink.testutils.junit.RetryOnFailure;
 import org.apache.flink.testutils.junit.RetryRule;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.PoisonPill;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
@@ -77,11 +78,6 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
@@ -93,20 +89,29 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLongArray;
 import java.util.concurrent.atomic.AtomicReference;
 
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob;
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
+/**
+ * Test JobManager recovery.
+ */
 public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 	@Rule
 	public RetryRule retryRule = new RetryRule();
 
-	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+	private static final ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
 
-	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+	private static final FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
 
 	private static final File FileStateBackendBasePath;
 
@@ -147,16 +152,16 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 	private static final int Parallelism = 8;
 
-	private static CountDownLatch CompletedCheckpointsLatch = new CountDownLatch(4);
-	private static CountDownLatch CompletedCheckpointsLatch2 = new CountDownLatch(6);
+	private static CountDownLatch completedCheckpointsLatch = new CountDownLatch(4);
+	private static CountDownLatch completedCheckpointsLatch2 = new CountDownLatch(6);
 
-	private static AtomicLongArray RecoveredStates = new AtomicLongArray(Parallelism);
+	private static AtomicLongArray recoveredStates = new AtomicLongArray(Parallelism);
 
-	private static CountDownLatch FinalCountLatch = new CountDownLatch(1);
+	private static CountDownLatch finalCountLatch = new CountDownLatch(1);
 
-	private static AtomicReference<Long> FinalCount = new AtomicReference<>();
+	private static AtomicReference<Long> finalCount = new AtomicReference<>();
 
-	private static long LastElement = -1;
+	private static long lastElement = -1;
 
 	private static final int retainedCheckpoints = 2;
 
@@ -166,7 +171,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 	 * @see <a href="https://issues.apache.org/jira/browse/FLINK-3185">FLINK-3185</a>
 	 */
 	@Test
-	@RetryOnFailure(times=1)
+	@RetryOnFailure(times = 1)
 	public void testCheckpointRecoveryFailure() throws Exception {
 		final Deadline testDeadline = TestTimeOut.fromNow();
 		final String zooKeeperQuorum = ZooKeeper.getConnectString();
@@ -361,7 +366,6 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
 			config.setString(CoreOptions.CHECKPOINTS_DIRECTORY, temporaryFolder.newFolder().toString());
 
-
 			String tmpFolderString = temporaryFolder.newFolder().toString();
 			config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, tmpFolderString);
 			config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
@@ -399,17 +403,17 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 			JobGraph jobGraph = env.getStreamGraph().getJobGraph();
 			miniCluster.submitJobDetached(jobGraph);
 
-			CompletedCheckpointsLatch.await();
+			completedCheckpointsLatch.await();
 
 			jmGateway.tell(PoisonPill.getInstance());
 
 			// Wait to finish
-			FinalCountLatch.await();
+			finalCountLatch.await();
 
-			assertEquals(expectedSum, (long) FinalCount.get());
+			assertEquals(expectedSum, (long) finalCount.get());
 
 			for (int i = 0; i < Parallelism; i++) {
-				assertNotEquals(0, RecoveredStates.get(i));
+				assertNotEquals(0, recoveredStates.get(i));
 			}
 
 		} finally {
@@ -467,7 +471,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 				synchronized (ctx.getCheckpointLock()) {
 					if (current <= end) {
 						ctx.collect(current++);
-					} else if(repeat > 0) {
+					} else if (repeat > 0) {
 						--repeat;
 						current = 0;
 					} else {
@@ -481,9 +485,9 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 				}
 			}
 
-			CompletedCheckpointsLatch2.await();
+			completedCheckpointsLatch2.await();
 			synchronized (ctx.getCheckpointLock()) {
-				ctx.collect(LastElement);
+				ctx.collect(lastElement);
 			}
 		}
 
@@ -503,7 +507,7 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 
 			// This is necessary to make sure that something is recovered at all. Otherwise it
 			// might happen that the job is restarted from the beginning.
-			RecoveredStates.set(getRuntimeContext().getIndexOfThisSubtask(), 1);
+			recoveredStates.set(getRuntimeContext().getIndexOfThisSubtask(), 1);
 
 			sync.countDown();
 
@@ -535,12 +539,12 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		@Override
 		public void invoke(Long value) throws Exception {
 
-			if (value == LastElement) {
+			if (value == lastElement) {
 				numberOfReceivedLastElements++;
 
 				if (numberOfReceivedLastElements == Parallelism) {
-					FinalCount.set(current);
-					FinalCountLatch.countDown();
+					finalCount.set(current);
+					finalCountLatch.countDown();
 				}
 				else if (numberOfReceivedLastElements > Parallelism) {
 					throw new IllegalStateException("Received more elements than parallelism.");
@@ -572,12 +576,12 @@ public class JobManagerHACheckpointRecoveryITCase extends TestLogger {
 		@Override
 		public void notifyCheckpointComplete(long checkpointId) throws Exception {
 			LOG.debug("Checkpoint {} completed.", checkpointId);
-			CompletedCheckpointsLatch.countDown();
-			CompletedCheckpointsLatch2.countDown();
+			completedCheckpointsLatch.countDown();
+			completedCheckpointsLatch2.countDown();
 		}
 	}
 
-	public static class StatefulFlatMap extends RichFlatMapFunction<Long, Long> implements CheckpointedFunction {
+	private static class StatefulFlatMap extends RichFlatMapFunction<Long, Long> implements CheckpointedFunction {
 
 		private static final long serialVersionUID = 9031079547885320663L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
index 3f2eea3..44b27cd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java
@@ -18,13 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.actor.Props;
-import akka.actor.UntypedActor;
-import akka.testkit.TestActorRef;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.filefilter.TrueFileFilter;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.configuration.ConfigConstants;
@@ -55,19 +48,20 @@ import org.apache.flink.runtime.testutils.JobManagerProcess;
 import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.Props;
+import akka.actor.UntypedActor;
+import akka.testkit.TestActorRef;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
 import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
-import scala.Option;
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.util.Collection;
@@ -77,6 +71,14 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+import scala.Option;
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -85,9 +87,9 @@ import static org.junit.Assert.fail;
  */
 public class JobManagerHAJobGraphRecoveryITCase extends TestLogger {
 
-	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+	private static final ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
 
-	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+	private static final FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
 
 	@Rule
 	public TemporaryFolder tempFolder = new TemporaryFolder();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
index 2983d66..2820dd2 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -49,14 +46,15 @@ import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import scala.Option;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.IOException;
@@ -65,6 +63,10 @@ import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
+import scala.Option;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.apache.flink.runtime.testutils.CommonTestUtils.createTempDirectory;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -88,9 +90,9 @@ import static org.junit.Assert.fail;
 @RunWith(Parameterized.class)
 public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 
-	private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
+	private static final ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1);
 
-	private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
+	private static final FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES);
 
 	private static final File FileStateBackendBasePath;
 
@@ -164,8 +166,8 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 		env.getConfig().setExecutionMode(executionMode);
 		env.getConfig().disableSysoutLogging();
 
-		final long NUM_ELEMENTS = 100000L;
-		final DataSet<Long> result = env.generateSequence(1, NUM_ELEMENTS)
+		final long numElements = 100000L;
+		final DataSet<Long> result = env.generateSequence(1, numElements)
 				// make sure every mapper is involved (no one is skipped because of lazy split assignment)
 				.rebalance()
 				// the majority of the behavior is in the MapFunction
@@ -209,7 +211,7 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
 				.flatMap(new RichFlatMapFunction<Long, Long>() {
 					@Override
 					public void flatMap(Long value, Collector<Long> out) throws Exception {
-						assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, (long) value);
+						assertEquals(numElements * (numElements + 1L) / 2L, (long) value);
 
 						int taskIndex = getRuntimeContext().getIndexOfThisSubtask();
 						AbstractTaskManagerProcessFailureRecoveryTest.touchFile(

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
index 59d5a51..b42204d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.recovery;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -42,15 +37,13 @@ import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.util.NetUtils;
-
 import org.apache.flink.util.TestLogger;
-import org.junit.Test;
 
-import scala.Some;
-import scala.Tuple2;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import org.junit.Test;
 
 import java.io.File;
 import java.io.StringWriter;
@@ -58,6 +51,12 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import scala.Some;
+import scala.Tuple2;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath;
 import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath;
 import static org.junit.Assert.assertFalse;
@@ -71,7 +70,7 @@ import static org.junit.Assert.fail;
  */
 @SuppressWarnings("serial")
 public class ProcessFailureCancelingITCase extends TestLogger {
-	
+
 	@Test
 	public void testCancelingOnProcessFailure() throws Exception {
 		final StringWriter processOutput = new StringWriter();
@@ -137,14 +136,14 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			// start the first two TaskManager processes
 			taskManagerProcess = new ProcessBuilder(command).start();
 			new CommonTestUtils.PipeForwarder(taskManagerProcess.getErrorStream(), processOutput);
-			
+
 			// we wait for the JobManager to have the two TaskManagers available
 			// since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes)
 			waitUntilNumTaskManagersAreRegistered(jmActor, 1, 120000);
-			
+
 			final Throwable[] errorRef = new Throwable[1];
 
-			// start the test program, which infinitely blocks 
+			// start the test program, which infinitely blocks
 			Runnable programRunner = new Runnable() {
 				@Override
 				public void run() {
@@ -176,7 +175,7 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 					}
 				}
 			};
-			
+
 			Thread programThread = new Thread(programRunner);
 
 			// kill the TaskManager
@@ -186,21 +185,21 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			// immediately submit the job. this should hit the case
 			// where the JobManager still thinks it has the TaskManager and tries to send it tasks
 			programThread.start();
-			
+
 			// try to cancel the job
 			cancelRunningJob(jmActor);
 
 			// we should see a failure within reasonable time (10s is the ask timeout).
-			// since the CI environment is often slow, we conservatively give it up to 2 minutes, 
+			// since the CI environment is often slow, we conservatively give it up to 2 minutes,
 			// to fail, which is much lower than the failure time given by the heartbeats ( > 2000s)
-			
+
 			programThread.join(120000);
-			
+
 			assertFalse("The program did not cancel in time (2 minutes)", programThread.isAlive());
-			
+
 			Throwable error = errorRef[0];
 			assertNotNull("The program did not fail properly", error);
-			
+
 			assertTrue(error instanceof ProgramInvocationException);
 			// all seems well :-)
 		}
@@ -225,15 +224,15 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			}
 		}
 	}
-	
+
 	private void cancelRunningJob(ActorRef jobManager) throws Exception {
 		final FiniteDuration askTimeout = new FiniteDuration(10, TimeUnit.SECONDS);
-		
+
 		// try at most for 30 seconds
 		final long deadline = System.currentTimeMillis() + 30000;
 
 		JobID jobId = null;
-		
+
 		do {
 			Future<Object> response = Patterns.ask(jobManager,
 					JobManagerMessages.getRequestRunningJobsStatus(), new Timeout(askTimeout));
@@ -247,9 +246,9 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			}
 
 			if (result instanceof JobManagerMessages.RunningJobsStatus) {
-	
+
 				List<JobStatusMessage> jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
-				
+
 				if (jobs.size() == 1) {
 					jobId = jobs.get(0).getJobId();
 					break;
@@ -262,7 +261,7 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 			// we never found it running, must have failed already
 			return;
 		}
-		
+
 		// tell the JobManager to cancel the job
 		jobManager.tell(
 			new JobManagerMessages.LeaderSessionMessage(
@@ -272,8 +271,7 @@ public class ProcessFailureCancelingITCase extends TestLogger {
 	}
 
 	private void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelay)
-			throws Exception
-	{
+			throws Exception {
 		final long deadline = System.currentTimeMillis() + maxDelay;
 		while (true) {
 			long remaining = deadline - System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
index a0d6b58..6fc876f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
@@ -21,8 +21,12 @@ package org.apache.flink.test.recovery;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
 import org.junit.BeforeClass;
 
+/**
+ * Test cluster configuration with failure-rate recovery.
+ */
 public class SimpleRecoveryFailureRateStrategyITBase extends SimpleRecoveryITCaseBase {
 	@BeforeClass
 	public static void setupCluster() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
index f09efc5..405a1c7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
@@ -21,8 +21,12 @@ package org.apache.flink.test.recovery;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
 import org.junit.BeforeClass;
 
+/**
+ * Test cluster configuration with fixed-delay recovery.
+ */
 public class SimpleRecoveryFixedDelayRestartStrategyITBase extends SimpleRecoveryITCaseBase {
 	@BeforeClass
 	public static void setupCluster() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
index bf7c524..ceef78b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
@@ -27,13 +27,17 @@ import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
 import org.junit.AfterClass;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * A series of tests (reusing one FlinkMiniCluster) where tasks fail (one or more time)
@@ -130,14 +134,14 @@ public abstract class SimpleRecoveryITCaseBase {
 
 	private void executeAndRunAssertions(ExecutionEnvironment env) throws Exception {
 		try {
-            JobExecutionResult result = env.execute();
-            assertTrue(result.getNetRuntime() >= 0);
-            assertNotNull(result.getAllAccumulatorResults());
-            assertTrue(result.getAllAccumulatorResults().isEmpty());
-        }
-        catch (JobExecutionException e) {
-            fail("The program should have succeeded on the second run");
-        }
+			JobExecutionResult result = env.execute();
+			assertTrue(result.getNetRuntime() >= 0);
+			assertNotNull(result.getAllAccumulatorResults());
+			assertTrue(result.getAllAccumulatorResults().isEmpty());
+		}
+		catch (JobExecutionException e) {
+			fail("The program should have succeeded on the second run");
+		}
 	}
 
 	@Test


[09/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java
new file mode 100644
index 0000000..2452475
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java
@@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.distributions.DataDistribution;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.utils.DataSetUtils;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.TestBaseUtils;
+import org.apache.flink.test.util.TestEnvironment;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Integration tests for custom {@link DataDistribution}.
+ */
+@SuppressWarnings("serial")
+public class CustomDistributionITCase extends TestLogger {
+
+	// ------------------------------------------------------------------------
+	//  The mini cluster that is shared across tests
+	// ------------------------------------------------------------------------
+
+	private static LocalFlinkMiniCluster cluster;
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		cluster = TestBaseUtils.startCluster(1, 8, false, false, true);
+	}
+
+	@AfterClass
+	public static void teardown() throws Exception {
+		TestBaseUtils.stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
+	}
+
+	@Before
+	public void prepare() {
+		TestEnvironment clusterEnv = new TestEnvironment(cluster, 1, false);
+		clusterEnv.setAsContext();
+	}
+
+	@After
+	public void cleanup() {
+		TestEnvironment.unsetAsContext();
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Test the record partitioned rightly with one field according to the customized data distribution.
+	 */
+	@Test
+	public void testPartitionWithDistribution1() throws Exception {
+		final TestDataDist1 dist = new TestDataDist1();
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(dist.getParallelism());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Boolean> result = DataSetUtils
+			.partitionByRange(input, dist, 0)
+			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Long, String>, Boolean>() {
+
+				@Override
+				public void mapPartition(Iterable<Tuple3<Integer, Long, String>> values, Collector<Boolean> out) throws Exception {
+					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
+
+					for (Tuple3<Integer, Long, String> s : values) {
+						boolean correctlyPartitioned = true;
+						if (pIdx == 0) {
+							Integer[] upper = dist.boundaries[0];
+							if (s.f0.compareTo(upper[0]) > 0) {
+								correctlyPartitioned = false;
+							}
+						}
+						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							Integer[] upper = dist.boundaries[pIdx];
+							if (s.f0.compareTo(upper[0]) > 0 || (s.f0.compareTo(lower[0]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+						else {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							if ((s.f0.compareTo(lower[0]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+
+						if (!correctlyPartitioned) {
+							fail("Record was not correctly partitioned: " + s.toString());
+						}
+					}
+				}
+			}
+			);
+
+		result.output(new DiscardingOutputFormat<Boolean>());
+		env.execute();
+	}
+
+	/**
+	 * Test the record partitioned rightly with two fields according to the customized data distribution.
+	 */
+	@Test
+	public void testRangeWithDistribution2() throws Exception {
+		final TestDataDist2 dist = new TestDataDist2();
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(dist.getParallelism());
+
+		DataSet<Tuple3<Integer, Integer, String>> input = env.fromElements(
+						new Tuple3<>(1, 5, "Hi"),
+						new Tuple3<>(1, 6, "Hi"),
+						new Tuple3<>(1, 7, "Hi"),
+						new Tuple3<>(1, 11, "Hello"),
+						new Tuple3<>(2, 3, "World"),
+						new Tuple3<>(2, 4, "World"),
+						new Tuple3<>(2, 5, "World"),
+						new Tuple3<>(2, 13, "Hello World"),
+						new Tuple3<>(3, 8, "Say"),
+						new Tuple3<>(4, 0, "Why"),
+						new Tuple3<>(4, 2, "Java"),
+						new Tuple3<>(4, 11, "Say Hello"),
+						new Tuple3<>(5, 1, "Hi Java!"),
+						new Tuple3<>(5, 2, "Hi Java?"),
+						new Tuple3<>(5, 3, "Hi Java again")
+			);
+
+		DataSet<Boolean> result = DataSetUtils
+			.partitionByRange(input, dist, 0, 1)
+			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Integer, String>, Boolean>() {
+
+				@Override
+				public void mapPartition(Iterable<Tuple3<Integer, Integer, String>> values, Collector<Boolean> out) throws Exception {
+					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
+					boolean correctlyPartitioned = true;
+
+					for (Tuple3<Integer, Integer, String> s : values) {
+
+						if (pIdx == 0) {
+							Integer[] upper = dist.boundaries[0];
+							if (s.f0.compareTo(upper[0]) > 0 ||
+								(s.f0.compareTo(upper[0]) == 0 && s.f1.compareTo(upper[1]) > 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							Integer[] upper = dist.boundaries[pIdx];
+
+							if (s.f0.compareTo(upper[0]) > 0 ||
+								(s.f0.compareTo(upper[0]) == 0 && s.f1.compareTo(upper[1]) > 0) ||
+								(s.f0.compareTo(lower[0]) < 0) ||
+								(s.f0.compareTo(lower[0]) == 0 && s.f1.compareTo(lower[1]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+						else {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							if ((s.f0.compareTo(lower[0]) < 0) ||
+								(s.f0.compareTo(lower[0]) == 0 && s.f1.compareTo(lower[1]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+
+						if (!correctlyPartitioned) {
+							fail("Record was not correctly partitioned: " + s.toString());
+						}
+					}
+				}
+			}
+			);
+
+		result.output(new DiscardingOutputFormat<Boolean>());
+		env.execute();
+	}
+
+	/*
+	 * Test the number of partition keys less than the number of distribution fields
+	 */
+	@Test
+	public void testPartitionKeyLessDistribution() throws Exception {
+		final TestDataDist2 dist = new TestDataDist2();
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(dist.getParallelism());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Boolean> result = DataSetUtils
+			.partitionByRange(input, dist, 0)
+			.mapPartition(new RichMapPartitionFunction<Tuple3<Integer, Long, String>, Boolean>() {
+
+				@Override
+				public void mapPartition(Iterable<Tuple3<Integer, Long, String>> values, Collector<Boolean> out) throws Exception {
+					int pIdx = getRuntimeContext().getIndexOfThisSubtask();
+
+					for (Tuple3<Integer, Long, String> s : values) {
+						boolean correctlyPartitioned = true;
+						if (pIdx == 0) {
+							Integer[] upper = dist.boundaries[0];
+							if (s.f0.compareTo(upper[0]) > 0) {
+								correctlyPartitioned = false;
+							}
+						}
+						else if (pIdx > 0 && pIdx < dist.getParallelism() - 1) {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							Integer[] upper = dist.boundaries[pIdx];
+							if (s.f0.compareTo(upper[0]) > 0 || (s.f0.compareTo(lower[0]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+						else {
+							Integer[] lower = dist.boundaries[pIdx - 1];
+							if ((s.f0.compareTo(lower[0]) <= 0)) {
+								correctlyPartitioned = false;
+							}
+						}
+
+						if (!correctlyPartitioned) {
+							fail("Record was not correctly partitioned: " + s.toString());
+						}
+					}
+				}
+			}
+			);
+
+		result.output(new DiscardingOutputFormat<Boolean>());
+		env.execute();
+	}
+
+	/*
+	 * Test the number of partition keys larger than the number of distribution fields
+	 */
+	@Test(expected = IllegalArgumentException.class)
+	public void testPartitionMoreThanDistribution() throws Exception {
+		final TestDataDist2 dist = new TestDataDist2();
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		DataSetUtils.partitionByRange(input, dist, 0, 1, 2);
+	}
+
+	/**
+	 * The class is used to do the tests of range partition with one key.
+	 */
+	public static class TestDataDist1 implements DataDistribution {
+
+		public Integer[][] boundaries = new Integer[][]{
+			new Integer[]{4},
+			new Integer[]{9},
+			new Integer[]{13},
+			new Integer[]{18}
+		};
+
+		public TestDataDist1() {}
+
+		public int getParallelism() {
+			return boundaries.length;
+		}
+
+		@Override
+		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
+			return boundaries[bucketNum];
+		}
+
+		@Override
+		public int getNumberOfFields() {
+			return 1;
+		}
+
+		@Override
+		public TypeInformation[] getKeyTypes() {
+			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO};
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {}
+
+		@Override
+		public void read(DataInputView in) throws IOException {}
+	}
+
+	/**
+	 * The class is used to do the tests of range partition with two keys.
+	 */
+	public static class TestDataDist2 implements DataDistribution {
+
+		public Integer[][] boundaries = new Integer[][]{
+			new Integer[]{1, 6},
+			new Integer[]{2, 4},
+			new Integer[]{3, 9},
+			new Integer[]{4, 1},
+			new Integer[]{5, 2}
+		};
+
+		public TestDataDist2() {}
+
+		public int getParallelism() {
+			return boundaries.length;
+		}
+
+		@Override
+		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
+			return boundaries[bucketNum];
+		}
+
+		@Override
+		public int getNumberOfFields() {
+			return 2;
+		}
+
+		@Override
+		public TypeInformation[] getKeyTypes() {
+			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO};
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {}
+
+		@Override
+		public void read(DataInputView in) throws IOException {}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java
new file mode 100644
index 0000000..deb5170
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSinkITCase.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.BufferedReader;
+import java.util.Random;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for data sinks.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class DataSinkITCase extends MultipleProgramsTestBase {
+
+	public DataSinkITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private String resultPath;
+
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Before
+	public void before() throws Exception{
+		resultPath = tempFolder.newFile().toURI().toString();
+	}
+
+	@Test
+	public void testIntSortingParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ds = CollectionDataSets.getIntegerDataSet(env);
+		ds.writeAsText(resultPath).sortLocalOutput("*", Order.DESCENDING).setParallelism(1);
+
+		env.execute();
+
+		String expected = "5\n5\n5\n5\n5\n4\n4\n4\n4\n3\n3\n3\n2\n2\n1\n";
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+
+	}
+
+	@Test
+	public void testStringSortingParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		ds.writeAsText(resultPath).sortLocalOutput("*", Order.ASCENDING).setParallelism(1);
+
+		env.execute();
+
+		String expected = "Hello\n" +
+				"Hello world\n" +
+				"Hello world, how are you?\n" +
+				"Hi\n" +
+				"I am fine.\n" +
+				"LOL\n" +
+				"Luke Skywalker\n" +
+				"Random comment\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testTupleSortingSingleAscParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.ASCENDING).setParallelism(1);
+
+		env.execute();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n" +
+				"6,3,Luke Skywalker\n" +
+				"7,4,Comment#1\n" +
+				"8,4,Comment#2\n" +
+				"9,4,Comment#3\n" +
+				"10,4,Comment#4\n" +
+				"11,5,Comment#5\n" +
+				"12,5,Comment#6\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" +
+				"15,5,Comment#9\n" +
+				"16,6,Comment#10\n" +
+				"17,6,Comment#11\n" +
+				"18,6,Comment#12\n" +
+				"19,6,Comment#13\n" +
+				"20,6,Comment#14\n" +
+				"21,6,Comment#15\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testTupleSortingSingleDescParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		ds.writeAsCsv(resultPath).sortLocalOutput(0, Order.DESCENDING).setParallelism(1);
+
+		env.execute();
+
+		String expected = "21,6,Comment#15\n" +
+				"20,6,Comment#14\n" +
+				"19,6,Comment#13\n" +
+				"18,6,Comment#12\n" +
+				"17,6,Comment#11\n" +
+				"16,6,Comment#10\n" +
+				"15,5,Comment#9\n" +
+				"14,5,Comment#8\n" +
+				"13,5,Comment#7\n" +
+				"12,5,Comment#6\n" +
+				"11,5,Comment#5\n" +
+				"10,4,Comment#4\n" +
+				"9,4,Comment#3\n" +
+				"8,4,Comment#2\n" +
+				"7,4,Comment#1\n" +
+				"6,3,Luke Skywalker\n" +
+				"5,3,I am fine.\n" +
+				"4,3,Hello world, how are you?\n" +
+				"3,2,Hello world\n" +
+				"2,2,Hello\n" +
+				"1,1,Hi\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testTupleSortingDualParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		ds.writeAsCsv(resultPath)
+			.sortLocalOutput(1, Order.DESCENDING).sortLocalOutput(0, Order.ASCENDING)
+			.setParallelism(1);
+
+		env.execute();
+
+		String expected = "16,6,Comment#10\n" +
+				"17,6,Comment#11\n" +
+				"18,6,Comment#12\n" +
+				"19,6,Comment#13\n" +
+				"20,6,Comment#14\n" +
+				"21,6,Comment#15\n" +
+				"11,5,Comment#5\n" +
+				"12,5,Comment#6\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" +
+				"15,5,Comment#9\n" +
+				"7,4,Comment#1\n" +
+				"8,4,Comment#2\n" +
+				"9,4,Comment#3\n" +
+				"10,4,Comment#4\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n" +
+				"6,3,Luke Skywalker\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"1,1,Hi\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testTupleSortingNestedParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> ds =
+				CollectionDataSets.getGroupSortedNestedTupleDataSet2(env);
+		ds.writeAsText(resultPath)
+			.sortLocalOutput("f0.f1", Order.ASCENDING)
+			.sortLocalOutput("f1", Order.DESCENDING)
+			.setParallelism(1);
+
+		env.execute();
+
+		String expected =
+				"((2,1),a,3)\n" +
+				"((2,2),b,4)\n" +
+				"((1,2),a,1)\n" +
+				"((3,3),c,5)\n" +
+				"((1,3),a,2)\n" +
+				"((3,6),c,6)\n" +
+				"((4,9),c,7)\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testTupleSortingNestedParallelism1_2() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Tuple2<Integer, Integer>, String, Integer>> ds =
+				CollectionDataSets.getGroupSortedNestedTupleDataSet2(env);
+		ds.writeAsText(resultPath)
+			.sortLocalOutput(1, Order.ASCENDING)
+			.sortLocalOutput(2, Order.DESCENDING)
+			.setParallelism(1);
+
+		env.execute();
+
+		String expected =
+				"((2,1),a,3)\n" +
+				"((1,3),a,2)\n" +
+				"((1,2),a,1)\n" +
+				"((2,2),b,4)\n" +
+				"((4,9),c,7)\n" +
+				"((3,6),c,6)\n" +
+				"((3,3),c,5)\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testPojoSortingSingleParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
+		ds.writeAsText(resultPath).sortLocalOutput("number", Order.ASCENDING).setParallelism(1);
+
+		env.execute();
+
+		String expected = "1 First (10,100,1000,One) 10100\n" +
+				"2 First_ (10,105,1000,One) 10200\n" +
+				"3 First (11,102,3000,One) 10200\n" +
+				"4 First_ (11,106,1000,One) 10300\n" +
+				"5 First (11,102,2000,One) 10100\n" +
+				"6 Second_ (20,200,2000,Two) 10100\n" +
+				"7 Third (31,301,2000,Three) 10200\n" +
+				"8 Third_ (30,300,1000,Three) 10100\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testPojoSortingDualParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
+		ds.writeAsText(resultPath)
+			.sortLocalOutput("str", Order.ASCENDING)
+			.sortLocalOutput("number", Order.DESCENDING)
+			.setParallelism(1);
+
+		env.execute();
+
+		String expected =
+				"5 First (11,102,2000,One) 10100\n" +
+				"3 First (11,102,3000,One) 10200\n" +
+				"1 First (10,100,1000,One) 10100\n" +
+				"4 First_ (11,106,1000,One) 10300\n" +
+				"2 First_ (10,105,1000,One) 10200\n" +
+				"6 Second_ (20,200,2000,Two) 10100\n" +
+				"7 Third (31,301,2000,Three) 10200\n" +
+				"8 Third_ (30,300,1000,Three) 10100\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+
+	}
+
+	@Test
+	public void testPojoSortingNestedParallelism1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CollectionDataSets.POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
+		ds.writeAsText(resultPath)
+			.sortLocalOutput("nestedTupleWithCustom.f0", Order.ASCENDING)
+			.sortLocalOutput("nestedTupleWithCustom.f1.myInt", Order.DESCENDING)
+			.sortLocalOutput("nestedPojo.longNumber", Order.ASCENDING)
+			.setParallelism(1);
+
+		env.execute();
+
+		String expected =
+				"2 First_ (10,105,1000,One) 10200\n" +
+				"1 First (10,100,1000,One) 10100\n" +
+				"4 First_ (11,106,1000,One) 10300\n" +
+				"5 First (11,102,2000,One) 10100\n" +
+				"3 First (11,102,3000,One) 10200\n" +
+				"6 Second_ (20,200,2000,Two) 10100\n" +
+				"8 Third_ (30,300,1000,Three) 10100\n" +
+				"7 Third (31,301,2000,Three) 10200\n";
+
+		compareResultsByLinesInMemoryWithStrictOrder(expected, resultPath);
+	}
+
+	@Test
+	public void testSortingParallelism4() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Long> ds = env.generateSequence(0, 1000);
+		// randomize
+		ds.map(new MapFunction<Long, Long>() {
+
+			Random rand = new Random(1234L);
+			@Override
+			public Long map(Long value) throws Exception {
+				return rand.nextLong();
+			}
+		}).writeAsText(resultPath)
+			.sortLocalOutput("*", Order.ASCENDING)
+			.setParallelism(4);
+
+		env.execute();
+
+		BufferedReader[] resReaders = getResultReader(resultPath);
+		for (BufferedReader br : resReaders) {
+			long cmp = Long.MIN_VALUE;
+			while (br.ready()) {
+				long cur = Long.parseLong(br.readLine());
+				assertTrue("Invalid order of sorted output", cmp <= cur);
+				cmp = cur;
+			}
+			br.close();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java
new file mode 100644
index 0000000..f1a3f08
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/DataSourceITCase.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.TextInputFormat;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import org.junit.Assert;
+
+import java.util.List;
+
+/**
+ * Tests for the DataSource.
+ */
+public class DataSourceITCase extends JavaProgramTestBase {
+
+	private String inputPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		inputPath = createTempFile("input", "ab\n"
+				+ "cd\n"
+				+ "ef\n");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		/*
+		 * Test passing a configuration object to an input format
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		Configuration ifConf = new Configuration();
+		ifConf.setString("prepend", "test");
+
+		DataSet<String> ds = env.createInput(new TestInputFormat(new Path(inputPath))).withParameters(ifConf);
+		List<String> result = ds.collect();
+
+		String expectedResult = "ab\n"
+				+ "cd\n"
+				+ "ef\n";
+
+		compareResultAsText(result, expectedResult);
+	}
+
+	private static class TestInputFormat extends TextInputFormat {
+		private static final long serialVersionUID = 1L;
+
+		public TestInputFormat(Path filePath) {
+			super(filePath);
+		}
+
+		@Override
+		public void configure(Configuration parameters) {
+			super.configure(parameters);
+
+			Assert.assertNotNull(parameters.getString("prepend", null));
+			Assert.assertEquals("test", parameters.getString("prepend", null));
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java
new file mode 100644
index 0000000..1215660
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/DistinctITCase.java
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Integration tests for {@link DataSet#distinct}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class DistinctITCase extends MultipleProgramsTestBase {
+
+	public DistinctITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelector() throws Exception {
+		/*
+		 * check correctness of distinct on tuples with key field selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct(0, 1, 2);
+
+		List<Tuple3<Integer, Long, String>> result = distinctDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnTuplesWithKeyFieldSelectorWithNotAllFieldsSelected() throws Exception{
+		/*
+		 * check correctness of distinct on tuples with key field selector with not all fields selected
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple1<Integer>> distinctDs = ds.union(ds).distinct(0).project(0);
+
+		List<Tuple1<Integer>> result = distinctDs.collect();
+
+		String expected = "1\n" +
+				"2\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnTuplesWithKeyExtractorFunction() throws Exception {
+		/*
+		 * check correctness of distinct on tuples with key extractor function
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple1<Integer>> reduceDs = ds.union(ds).distinct(new KeySelector1()).project(0);
+
+		List<Tuple1<Integer>> result = reduceDs.collect();
+
+		String expected = "1\n" + "2\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector1 implements KeySelector<Tuple5<Integer, Long,  Integer, String, Long>, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(Tuple5<Integer, Long,  Integer, String, Long> in) {
+			return in.f0;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnCustomTypeWithTypeExtractor() throws Exception {
+		/*
+		 * check correctness of distinct on custom type with type extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<Tuple1<Integer>> reduceDs = ds
+				.distinct(new KeySelector3())
+				.map(new Mapper3());
+
+		List<Tuple1<Integer>> result = reduceDs.collect();
+
+		String expected = "1\n" +
+				"2\n" +
+				"3\n" +
+				"4\n" +
+				"5\n" +
+				"6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Mapper3 extends RichMapFunction<CustomType, Tuple1<Integer>> {
+		@Override
+		public Tuple1<Integer> map(CustomType value) throws Exception {
+			return new Tuple1<Integer>(value.myInt);
+		}
+	}
+
+	private static class KeySelector3 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnTuples() throws Exception{
+		/*
+		 * check correctness of distinct on tuples
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> distinctDs = ds.union(ds).distinct();
+
+		List<Tuple3<Integer, Long, String>> result = distinctDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnCustomTypeWithTupleReturningTypeExtractor() throws Exception{
+		/*
+		 * check correctness of distinct on custom type with tuple-returning type extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> reduceDs = ds
+				.distinct(new KeySelector2())
+				.project(0, 4);
+
+		List<Tuple2<Integer, Long>> result = reduceDs.collect();
+
+		String expected = "1,1\n" +
+				"2,1\n" +
+				"2,2\n" +
+				"3,2\n" +
+				"3,3\n" +
+				"4,1\n" +
+				"4,2\n" +
+				"5,1\n" +
+				"5,2\n" +
+				"5,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnTuplesWithFieldExpressions() throws Exception {
+		/*
+		 * check correctness of distinct on tuples with field expressions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple1<Integer>> reduceDs = ds.union(ds).distinct("f0").project(0);
+
+		List<Tuple1<Integer>> result = reduceDs.collect();
+
+		String expected = "1\n" +
+				"2\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnPojos() throws Exception {
+		/*
+		 * check correctness of distinct on Pojos
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
+		DataSet<Integer> reduceDs = ds.distinct("nestedPojo.longNumber").map(new Mapper2());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "10000\n20000\n30000\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper2 implements MapFunction<CollectionDataSets.POJO, Integer> {
+		@Override
+		public Integer map(POJO value) throws Exception {
+			return (int) value.nestedPojo.longNumber;
+		}
+	}
+
+	@Test
+	public void testDistinctOnFullPojo() throws Exception {
+		/*
+		 * distinct on full Pojo
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds = CollectionDataSets.getDuplicatePojoDataSet(env);
+		DataSet<Integer> reduceDs = ds.distinct().map(new Mapper1());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "10000\n20000\n30000\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Mapper1 implements MapFunction<CollectionDataSets.POJO, Integer> {
+		@Override
+		public Integer map(POJO value) throws Exception {
+			return (int) value.nestedPojo.longNumber;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnAtomic() throws Exception {
+		/*
+		 * check correctness of distinct on Integers
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Integer> ds = CollectionDataSets.getIntegerDataSet(env);
+		DataSet<Integer> reduceDs = ds.distinct();
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "1\n2\n3\n4\n5";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfDistinctOnAtomicWithSelectAllChar() throws Exception {
+		/*
+		 * check correctness of distinct on Strings, using Keys.ExpressionKeys.SELECT_ALL_CHAR
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> reduceDs = ds.union(ds).distinct("*");
+
+		List<String> result = reduceDs.collect();
+
+		String expected = "I am fine.\n" +
+				"Luke Skywalker\n" +
+				"LOL\n" +
+				"Hello world, how are you?\n" +
+				"Hi\n" +
+				"Hello world\n" +
+				"Hello\n" +
+				"Random comment\n";
+
+		compareResultAsText(result, expected);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java
new file mode 100644
index 0000000..f7f993b
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ExecutionEnvironmentITCase.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.api.common.io.GenericInputFormat;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test ExecutionEnvironment from user perspective.
+ */
+@SuppressWarnings("serial")
+public class ExecutionEnvironmentITCase extends TestLogger {
+
+	private static final int PARALLELISM = 5;
+
+	/**
+	 * Ensure that the user can pass a custom configuration object to the LocalEnvironment.
+	 */
+	@Test
+	public void testLocalEnvironmentWithConfig() throws Exception {
+		Configuration conf = new Configuration();
+		conf.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(conf);
+		env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
+		env.getConfig().disableSysoutLogging();
+
+		DataSet<Integer> result = env.createInput(new ParallelismDependentInputFormat())
+				.rebalance()
+				.mapPartition(new RichMapPartitionFunction<Integer, Integer>() {
+					@Override
+					public void mapPartition(Iterable<Integer> values, Collector<Integer> out) throws Exception {
+						out.collect(getRuntimeContext().getIndexOfThisSubtask());
+					}
+				});
+		List<Integer> resultCollection = result.collect();
+		assertEquals(PARALLELISM, resultCollection.size());
+	}
+
+	private static class ParallelismDependentInputFormat extends GenericInputFormat<Integer> {
+
+		private transient boolean emitted;
+
+		@Override
+		public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
+			assertEquals(PARALLELISM, numSplits);
+			return super.createInputSplits(numSplits);
+		}
+
+		@Override
+		public boolean reachedEnd() {
+			return emitted;
+		}
+
+		@Override
+		public Integer nextRecord(Integer reuse) {
+			if (emitted) {
+				return null;
+			}
+			emitted = true;
+			return 1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java
new file mode 100644
index 0000000..6fa2dce
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link FilterFunction} and {@link RichFilterFunction}.
+ */
+@RunWith(Parameterized.class)
+public class FilterITCase extends MultipleProgramsTestBase {
+	public FilterITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testAllRejectingFilter() throws Exception {
+		/*
+		 * Test all-rejecting filter.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new Filter1());
+
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Filter1 implements FilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return false;
+		}
+	}
+
+	@Test
+	public void testAllPassingFilter() throws Exception {
+		/*
+		 * Test all-passing filter.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new Filter2());
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n" +
+				"6,3,Luke Skywalker\n" +
+				"7,4,Comment#1\n" +
+				"8,4,Comment#2\n" +
+				"9,4,Comment#3\n" +
+				"10,4,Comment#4\n" +
+				"11,5,Comment#5\n" +
+				"12,5,Comment#6\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" +
+				"15,5,Comment#9\n" +
+				"16,6,Comment#10\n" +
+				"17,6,Comment#11\n" +
+				"18,6,Comment#12\n" +
+				"19,6,Comment#13\n" +
+				"20,6,Comment#14\n" +
+				"21,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Filter2 implements FilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return true;
+		}
+	}
+
+	@Test
+	public void testFilterOnStringTupleField() throws Exception {
+		/*
+		 * Test filter on String tuple field.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new Filter3());
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "3,2,Hello world\n"
+				+
+				"4,3,Hello world, how are you?\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	private static class Filter3 implements FilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return value.f2.contains("world");
+		}
+	}
+
+	@Test
+	public void testFilterOnIntegerTupleField() throws Exception {
+		/*
+		 * Test filter on Integer tuple field.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new Filter4());
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "2,2,Hello\n" +
+				"4,3,Hello world, how are you?\n" +
+				"6,3,Luke Skywalker\n" +
+				"8,4,Comment#2\n" +
+				"10,4,Comment#4\n" +
+				"12,5,Comment#6\n" +
+				"14,5,Comment#8\n" +
+				"16,6,Comment#10\n" +
+				"18,6,Comment#12\n" +
+				"20,6,Comment#14\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class Filter4 implements FilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return (value.f0 % 2) == 0;
+		}
+	}
+
+	@Test
+	public void testFilterBasicType() throws Exception {
+		/*
+		 * Test filter on basic type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> filterDs = ds.
+				filter(new Filter5());
+		List<String> result = filterDs.collect();
+
+		String expected = "Hi\n" +
+				"Hello\n" +
+				"Hello world\n" +
+				"Hello world, how are you?\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Filter5 implements FilterFunction<String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(String value) throws Exception {
+			return value.startsWith("H");
+		}
+	}
+
+	@Test
+	public void testFilterOnCustomType() throws Exception {
+		/*
+		 * Test filter on custom type
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> filterDs = ds.
+				filter(new Filter6());
+		List<CustomType> result = filterDs.collect();
+
+		String expected = "3,3,Hello world, how are you?\n"
+				+
+				"3,4,I am fine.\n" +
+				"3,5,Luke Skywalker\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class Filter6 implements FilterFunction<CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(CustomType value) throws Exception {
+			return value.myString.contains("a");
+		}
+	}
+
+	@Test
+	public void testRichFilterOnStringTupleField() throws Exception {
+		/*
+		 * Test filter on String tuple field.
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new RichFilter1()).withBroadcastSet(ints, "ints");
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichFilter1 extends RichFilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		int literal = -1;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			for (int i: ints) {
+				literal = literal < i ? i : literal;
+			}
+		}
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return value.f0 < literal;
+		}
+	}
+
+	@Test
+	public void testFilterWithBroadcastVariables() throws Exception {
+		/*
+		 * Test filter with broadcast variables
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> filterDs = ds.
+				filter(new RichFilter2()).withBroadcastSet(intDs, "ints");
+		List<Tuple3<Integer, Long, String>> result = filterDs.collect();
+
+		String expected = "11,5,Comment#5\n" +
+				"12,5,Comment#6\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" +
+				"15,5,Comment#9\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichFilter2 extends RichFilterFunction<Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private  int broadcastSum = 0;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			for (Integer i : ints) {
+				broadcastSum += i;
+			}
+		}
+
+		@Override
+		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
+			return (value.f1 == (broadcastSum / 11));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java
new file mode 100644
index 0000000..abbd446
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/FirstNITCase.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.operators;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.GroupReduceOperator;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+/**
+ * Integration tests for {@link DataSet#first}.
+ */
+@RunWith(Parameterized.class)
+public class FirstNITCase extends MultipleProgramsTestBase {
+	public FirstNITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testFirstNOnUngroupedDS() throws Exception {
+		/*
+		 * First-n on ungrouped data set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple1<Integer>> seven = ds.first(7).map(new OneMapper()).sum(0);
+
+		List<Tuple1<Integer>> result = seven.collect();
+
+		String expected = "(7)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testFirstNOnGroupedDS() throws Exception {
+		/*
+		 * First-n on grouped data set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Long, Integer>> first = ds.groupBy(1).first(4)
+				.map(new OneMapper2()).groupBy(0).sum(1);
+
+		List<Tuple2<Long, Integer>> result = first.collect();
+
+		String expected = "(1,1)\n(2,2)\n(3,3)\n(4,4)\n(5,4)\n(6,4)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testFirstNOnGroupedAndSortedDS() throws Exception {
+		/*
+		 * First-n on grouped and sorted data set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Long, Integer>> first = ds.groupBy(1).sortGroup(0, Order.DESCENDING).first(3)
+				.project(1, 0);
+
+		List<Tuple2<Long, Integer>> result = first.collect();
+
+		String expected = "(1,1)\n"
+				+ "(2,3)\n(2,2)\n"
+				+ "(3,6)\n(3,5)\n(3,4)\n"
+				+ "(4,10)\n(4,9)\n(4,8)\n"
+				+ "(5,15)\n(5,14)\n(5,13)\n"
+				+ "(6,21)\n(6,20)\n(6,19)\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	/**
+	 * Test for FLINK-2135.
+	 */
+	@Test
+	public void testFaultyCast() throws Exception {
+		ExecutionEnvironment ee = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> b = ee.fromElements("a", "b");
+		GroupReduceOperator<String, String> a = b.groupBy(new KeySelector<String, Long>() {
+			@Override
+			public Long getKey(String value) throws Exception {
+				return 1L;
+			}
+		}).sortGroup(new KeySelector<String, Double>() {
+			@Override
+			public Double getKey(String value) throws Exception {
+				return 1.0;
+			}
+		}, Order.DESCENDING).first(1);
+
+		List<String> result = b.collect();
+
+		String expected = "a\nb";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class OneMapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple1<Integer>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple1<Integer> one = new Tuple1<Integer>(1);
+		@Override
+		public Tuple1<Integer> map(Tuple3<Integer, Long, String> value) {
+			return one;
+		}
+	}
+
+	private static class OneMapper2 implements MapFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple2<Long, Integer> one = new Tuple2<>(0L, 1);
+
+		@Override
+		public Tuple2<Long, Integer> map(Tuple3<Integer, Long, String> value) {
+			one.f0 = value.f1;
+			return one;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java
new file mode 100644
index 0000000..4b683a9
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link FlatMapFunction} and {@link RichFlatMapFunction}.
+ */
+@RunWith(Parameterized.class)
+public class FlatMapITCase extends MultipleProgramsTestBase {
+	public FlatMapITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testNonPassingFlatMap() throws Exception {
+		/*
+		 * Test non-passing flatmap
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> nonPassingFlatMapDs = ds.
+				flatMap(new FlatMapper1());
+
+		List<String> result = nonPassingFlatMapDs.collect();
+
+		String expected = "\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class FlatMapper1 implements FlatMapFunction<String, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<String> out) throws Exception {
+			if (value.contains("bananas")) {
+				out.collect(value);
+			}
+		}
+	}
+
+	@Test
+	public void testDataDuplicatingFlatMap() throws Exception {
+		/*
+		 * Test data duplicating flatmap
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<String> ds = CollectionDataSets.getStringDataSet(env);
+		DataSet<String> duplicatingFlatMapDs = ds.
+				flatMap(new FlatMapper2());
+
+		List<String> result = duplicatingFlatMapDs.collect();
+
+		String expected = "Hi\n" + "HI\n" +
+				"Hello\n" + "HELLO\n" +
+				"Hello world\n" + "HELLO WORLD\n" +
+				"Hello world, how are you?\n" + "HELLO WORLD, HOW ARE YOU?\n" +
+				"I am fine.\n" + "I AM FINE.\n" +
+				"Luke Skywalker\n" + "LUKE SKYWALKER\n" +
+				"Random comment\n" + "RANDOM COMMENT\n" +
+				"LOL\n" + "LOL\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class FlatMapper2 implements FlatMapFunction<String, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<String> out) throws Exception {
+			out.collect(value);
+			out.collect(value.toUpperCase());
+		}
+	}
+
+	@Test
+	public void testFlatMapWithVaryingNumberOfEmittedTuples() throws Exception {
+		/*
+		 * Test flatmap with varying number of emitted tuples
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> varyingTuplesMapDs = ds.
+				flatMap(new FlatMapper3());
+
+		List<Tuple3<Integer, Long, String>> result = varyingTuplesMapDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" + "2,2,Hello\n" +
+				"4,3,Hello world, how are you?\n" +
+				"5,3,I am fine.\n" + "5,3,I am fine.\n" +
+				"7,4,Comment#1\n" +
+				"8,4,Comment#2\n" + "8,4,Comment#2\n" +
+				"10,4,Comment#4\n" +
+				"11,5,Comment#5\n" + "11,5,Comment#5\n" +
+				"13,5,Comment#7\n" +
+				"14,5,Comment#8\n" + "14,5,Comment#8\n" +
+				"16,6,Comment#10\n" +
+				"17,6,Comment#11\n" + "17,6,Comment#11\n" +
+				"19,6,Comment#13\n" +
+				"20,6,Comment#14\n" + "20,6,Comment#14\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class FlatMapper3 implements FlatMapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(Tuple3<Integer, Long, String> value,
+				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			final int numTuples = value.f0 % 3;
+			for (int i = 0; i < numTuples; i++) {
+				out.collect(value);
+			}
+		}
+	}
+
+	@Test
+	public void testTypeConversionFlatMapperCustomToTuple() throws Exception {
+		/*
+		 * Test type conversion flatmapper (Custom -> Tuple)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> typeConversionFlatMapDs = ds.
+				flatMap(new FlatMapper4());
+
+		List<Tuple3<Integer, Long, String>> result = typeConversionFlatMapDs.collect();
+
+		String expected = "1,0,Hi\n" +
+				"2,1,Hello\n" +
+				"2,2,Hello world\n" +
+				"3,3,Hello world, how are you?\n" +
+				"3,4,I am fine.\n" +
+				"3,5,Luke Skywalker\n" +
+				"4,6,Comment#1\n" +
+				"4,7,Comment#2\n" +
+				"4,8,Comment#3\n" +
+				"4,9,Comment#4\n" +
+				"5,10,Comment#5\n" +
+				"5,11,Comment#6\n" +
+				"5,12,Comment#7\n" +
+				"5,13,Comment#8\n" +
+				"5,14,Comment#9\n" +
+				"6,15,Comment#10\n" +
+				"6,16,Comment#11\n" +
+				"6,17,Comment#12\n" +
+				"6,18,Comment#13\n" +
+				"6,19,Comment#14\n" +
+				"6,20,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class FlatMapper4 implements FlatMapFunction<CustomType, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> outTuple =
+				new Tuple3<Integer, Long, String>();
+
+		@Override
+		public void flatMap(CustomType value, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			outTuple.setField(value.myInt, 0);
+			outTuple.setField(value.myLong, 1);
+			outTuple.setField(value.myString, 2);
+			out.collect(outTuple);
+		}
+	}
+
+	@Test
+	public void testTypeConversionFlatMapperTupleToBasic() throws Exception {
+		/*
+		 * Test type conversion flatmapper (Tuple -> Basic)
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<String> typeConversionFlatMapDs = ds.
+				flatMap(new FlatMapper5());
+
+		List<String> result = typeConversionFlatMapDs.collect();
+
+		String expected = "Hi\n" + "Hello\n" + "Hello world\n"
+				+
+				"Hello world, how are you?\n" +
+				"I am fine.\n" + "Luke Skywalker\n" +
+				"Comment#1\n" +	"Comment#2\n" +
+				"Comment#3\n" +	"Comment#4\n" +
+				"Comment#5\n" +	"Comment#6\n" +
+				"Comment#7\n" + "Comment#8\n" +
+				"Comment#9\n" +	"Comment#10\n" +
+				"Comment#11\n" + "Comment#12\n" +
+				"Comment#13\n" + "Comment#14\n" +
+				"Comment#15\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class FlatMapper5 implements FlatMapFunction<Tuple3<Integer, Long, String>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(Tuple3<Integer, Long, String> value, Collector<String> out) throws Exception {
+			out.collect(value.f2);
+		}
+	}
+
+	@Test
+	public void testFlatMapperIfUDFReturnsInputObjectMultipleTimesWhileChangingIt() throws Exception {
+		/*
+		 * Test flatmapper if UDF returns input object
+		 * multiple times and changes it in between
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> inputObjFlatMapDs = ds.
+				flatMap(new FlatMapper6());
+
+		List<Tuple3<Integer, Long, String>> result = inputObjFlatMapDs.collect();
+
+		String expected = "0,1,Hi\n" +
+				"0,2,Hello\n" + "1,2,Hello\n" +
+				"0,2,Hello world\n" + "1,2,Hello world\n" + "2,2,Hello world\n" +
+				"0,3,I am fine.\n" +
+				"0,3,Luke Skywalker\n" + "1,3,Luke Skywalker\n" +
+				"0,4,Comment#1\n" + "1,4,Comment#1\n" + "2,4,Comment#1\n" +
+				"0,4,Comment#3\n" +
+				"0,4,Comment#4\n" + "1,4,Comment#4\n" +
+				"0,5,Comment#5\n" + "1,5,Comment#5\n" + "2,5,Comment#5\n" +
+				"0,5,Comment#7\n" +
+				"0,5,Comment#8\n" + "1,5,Comment#8\n" +
+				"0,5,Comment#9\n" + "1,5,Comment#9\n" + "2,5,Comment#9\n" +
+				"0,6,Comment#11\n" +
+				"0,6,Comment#12\n" + "1,6,Comment#12\n" +
+				"0,6,Comment#13\n" + "1,6,Comment#13\n" + "2,6,Comment#13\n" +
+				"0,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class FlatMapper6 implements FlatMapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(Tuple3<Integer, Long, String> value,
+				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			final int numTuples = value.f0 % 4;
+			for (int i = 0; i < numTuples; i++) {
+				value.setField(i, 0);
+				out.collect(value);
+			}
+		}
+	}
+
+	@Test
+	public void testFlatMapWithBroadcastSet() throws Exception {
+		/*
+		 * Test flatmap with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ints = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> bcFlatMapDs = ds.
+				flatMap(new RichFlatMapper1()).withBroadcastSet(ints, "ints");
+		List<Tuple3<Integer, Long, String>> result = bcFlatMapDs.collect();
+
+		String expected = "55,1,Hi\n" +
+				"55,2,Hello\n" +
+				"55,2,Hello world\n" +
+				"55,3,Hello world, how are you?\n" +
+				"55,3,I am fine.\n" +
+				"55,3,Luke Skywalker\n" +
+				"55,4,Comment#1\n" +
+				"55,4,Comment#2\n" +
+				"55,4,Comment#3\n" +
+				"55,4,Comment#4\n" +
+				"55,5,Comment#5\n" +
+				"55,5,Comment#6\n" +
+				"55,5,Comment#7\n" +
+				"55,5,Comment#8\n" +
+				"55,5,Comment#9\n" +
+				"55,6,Comment#10\n" +
+				"55,6,Comment#11\n" +
+				"55,6,Comment#12\n" +
+				"55,6,Comment#13\n" +
+				"55,6,Comment#14\n" +
+				"55,6,Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class RichFlatMapper1 extends RichFlatMapFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private final Tuple3<Integer, Long, String> outTuple =
+				new Tuple3<Integer, Long, String>();
+		private Integer f2Replace = 0;
+
+		@Override
+		public void open(Configuration config) {
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			f2Replace = sum;
+		}
+
+		@Override
+		public void flatMap(Tuple3<Integer, Long, String> value,
+				Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			outTuple.setFields(f2Replace, value.f1, value.f2);
+			out.collect(outTuple);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java
new file mode 100644
index 0000000..f0ee031
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupCombineITCase.java
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.operators.UnsortedGrouping;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * The GroupCombine operator is not easy to test because it is essentially just a combiner. The result can be
+ * the result of a normal groupReduce at any stage its execution. The basic idea is to preserve the grouping key
+ * in the partial result, so that we can do a reduceGroup afterwards to finalize the results for verification.
+ * In addition, we can use hashPartition to partition the data and check if no shuffling (just combining) has
+ * been performed.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class GroupCombineITCase extends MultipleProgramsTestBase {
+
+	public GroupCombineITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	private static String identityResult = "1,1,Hi\n" +
+			"2,2,Hello\n" +
+			"3,2,Hello world\n" +
+			"4,3,Hello world, how are you?\n" +
+			"5,3,I am fine.\n" +
+			"6,3,Luke Skywalker\n" +
+			"7,4,Comment#1\n" +
+			"8,4,Comment#2\n" +
+			"9,4,Comment#3\n" +
+			"10,4,Comment#4\n" +
+			"11,5,Comment#5\n" +
+			"12,5,Comment#6\n" +
+			"13,5,Comment#7\n" +
+			"14,5,Comment#8\n" +
+			"15,5,Comment#9\n" +
+			"16,6,Comment#10\n" +
+			"17,6,Comment#11\n" +
+			"18,6,Comment#12\n" +
+			"19,6,Comment#13\n" +
+			"20,6,Comment#14\n" +
+			"21,6,Comment#15\n";
+
+	@Test
+	public void testAllGroupCombineIdentity() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
+				// combine
+				.combineGroup(new IdentityFunction())
+				// fully reduce
+				.reduceGroup(new IdentityFunction());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		compareResultAsTuples(result, identityResult);
+	}
+
+	@Test
+	public void testIdentity() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
+				// combine
+				.combineGroup(new IdentityFunction())
+				// fully reduce
+				.reduceGroup(new IdentityFunction());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		compareResultAsTuples(result, identityResult);
+	}
+
+	@Test
+	public void testIdentityWithGroupBy() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
+				.groupBy(1)
+				// combine
+				.combineGroup(new IdentityFunction())
+				// fully reduce
+				.reduceGroup(new IdentityFunction());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		compareResultAsTuples(result, identityResult);
+	}
+
+	@Test
+	public void testIdentityWithGroupByAndSort() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
+				.groupBy(1)
+				.sortGroup(1, Order.DESCENDING)
+				// reduce partially
+				.combineGroup(new IdentityFunction())
+				.groupBy(1)
+				.sortGroup(1, Order.DESCENDING)
+				// fully reduce
+				.reduceGroup(new IdentityFunction());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		compareResultAsTuples(result, identityResult);
+	}
+
+	@Test
+	public void testPartialReduceWithIdenticalInputOutputType() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// data
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple2<Long, Tuple3<Integer, Long, String>>> dsWrapped = ds
+				// wrap values as Kv pairs with the grouping key as key
+				.map(new Tuple3KvWrapper());
+
+		List<Tuple3<Integer, Long, String>> result = dsWrapped
+				.groupBy(0)
+				// reduce partially
+				.combineGroup(new Tuple3toTuple3GroupReduce())
+				.groupBy(0)
+				// reduce fully to check result
+				.reduceGroup(new Tuple3toTuple3GroupReduce())
+				//unwrap
+				.map(new MapFunction<Tuple2<Long, Tuple3<Integer, Long, String>>, Tuple3<Integer, Long, String>>() {
+					@Override
+					public Tuple3<Integer, Long, String> map(Tuple2<Long, Tuple3<Integer, Long, String>> value) throws Exception {
+						return value.f1;
+					}
+				}).collect();
+
+		String expected = "1,1,combined\n" +
+				"5,4,combined\n" +
+				"15,9,combined\n" +
+				"34,16,combined\n" +
+				"65,25,combined\n" +
+				"111,36,combined\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testPartialReduceWithDifferentInputOutputType() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// data
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		DataSet<Tuple2<Long, Tuple3<Integer, Long, String>>> dsWrapped = ds
+				// wrap values as Kv pairs with the grouping key as key
+				.map(new Tuple3KvWrapper());
+
+		List<Tuple2<Integer, Long>> result = dsWrapped
+				.groupBy(0)
+				// reduce partially
+				.combineGroup(new Tuple3toTuple2GroupReduce())
+				.groupBy(0)
+				// reduce fully to check result
+				.reduceGroup(new Tuple2toTuple2GroupReduce())
+				//unwrap
+				.map(new MapFunction<Tuple2<Long, Tuple2<Integer, Long>>, Tuple2<Integer, Long>>() {
+					@Override
+					public Tuple2<Integer, Long> map(Tuple2<Long, Tuple2<Integer, Long>> value) throws Exception {
+						return value.f1;
+					}
+				}).collect();
+
+		String expected = "1,3\n" +
+				"5,20\n" +
+				"15,58\n" +
+				"34,52\n" +
+				"65,70\n" +
+				"111,96\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	// check if no shuffle is being executed
+	public void testCheckPartitionShuffleGroupBy() throws Exception {
+
+		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		// data
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		// partition and group data
+		UnsortedGrouping<Tuple3<Integer, Long, String>> partitionedDS = ds.partitionByHash(0).groupBy(1);
+
+		List<Tuple2<Long, Integer>> result = partitionedDS
+				.combineGroup(
+						new GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>>() {
+			@Override
+			public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Long, Integer>> out) throws Exception {
+				int count = 0;
+				long key = 0;
+				for (Tuple3<Integer, Long, String> value : values) {
+					key = value.f1;
+					count++;
+				}
+				out.collect(new Tuple2<>(key, count));
+			}
+		}).collect();
+
+		String[] localExpected = new String[] { "(6,6)", "(5,5)" + "(4,4)", "(3,3)", "(2,2)", "(1,1)" };
+
+		String[] resultAsStringArray = new String[result.size()];
+		for (int i = 0; i < resultAsStringArray.length; ++i) {
+			resultAsStringArray[i] = result.get(i).toString();
+		}
+		Arrays.sort(resultAsStringArray);
+
+		Assert.assertEquals("The two arrays were identical.", false, Arrays.equals(localExpected, resultAsStringArray));
+	}
+
+	@Test
+	// check if parallelism of 1 results in the same data like a shuffle
+	public void testCheckPartitionShuffleDOP1() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		env.setParallelism(1);
+
+		// data
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		// partition and group data
+		UnsortedGrouping<Tuple3<Integer, Long, String>> partitionedDS = ds.partitionByHash(0).groupBy(1);
+
+		List<Tuple2<Long, Integer>> result = partitionedDS
+				.combineGroup(
+				new GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple2<Long, Integer>>() {
+					@Override
+					public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Long, Integer>> out) throws Exception {
+						int count = 0;
+						long key = 0;
+						for (Tuple3<Integer, Long, String> value : values) {
+							key = value.f1;
+							count++;
+						}
+						out.collect(new Tuple2<>(key, count));
+					}
+				}).collect();
+
+		String expected = "6,6\n" +
+				"5,5\n" +
+				"4,4\n" +
+				"3,3\n" +
+				"2,2\n" +
+				"1,1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	// check if all API methods are callable
+	public void testAPI() throws Exception {
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple1<String>> ds = CollectionDataSets.getStringDataSet(env).map(new MapFunction<String, Tuple1<String>>() {
+			@Override
+			public Tuple1<String> map(String value) throws Exception {
+				return new Tuple1<>(value);
+			}
+		});
+
+		// all methods on DataSet
+		ds.combineGroup(new GroupCombineFunctionExample())
+		.output(new DiscardingOutputFormat<Tuple1<String>>());
+
+		// all methods on UnsortedGrouping
+		ds.groupBy(0).combineGroup(new GroupCombineFunctionExample())
+		.output(new DiscardingOutputFormat<Tuple1<String>>());
+
+		// all methods on SortedGrouping
+		ds.groupBy(0).sortGroup(0, Order.ASCENDING).combineGroup(new GroupCombineFunctionExample())
+		.output(new DiscardingOutputFormat<Tuple1<String>>());
+
+		env.execute();
+	}
+
+	private static class GroupCombineFunctionExample implements GroupCombineFunction<Tuple1<String>, Tuple1<String>> {
+
+		@Override
+		public void combine(Iterable<Tuple1<String>> values, Collector<Tuple1<String>> out) throws Exception {
+			for (Tuple1<String> value : values) {
+				out.collect(value);
+			}
+		}
+	}
+
+	/**
+	 * For Scala GroupCombineITCase.
+	 */
+	public static class ScalaGroupCombineFunctionExample implements GroupCombineFunction<scala.Tuple1<String>, scala.Tuple1<String>> {
+
+		@Override
+		public void combine(Iterable<scala.Tuple1<String>> values, Collector<scala.Tuple1<String>> out) throws Exception {
+			for (scala.Tuple1<String> value : values) {
+				out.collect(value);
+			}
+		}
+	}
+
+	private static class IdentityFunction implements GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>,
+	GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+
+		@Override
+		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			for (Tuple3<Integer, Long, String> value : values) {
+				out.collect(new Tuple3<>(value.f0, value.f1, value.f2));
+			}
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			for (Tuple3<Integer, Long, String> value : values) {
+				out.collect(new Tuple3<>(value.f0, value.f1, value.f2));
+			}
+		}
+	}
+
+	private static class Tuple3toTuple3GroupReduce implements KvGroupReduce<Long, Tuple3<Integer, Long, String>,
+			Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+
+		@Override
+		public void combine(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values, Collector<Tuple2<Long,
+				Tuple3<Integer, Long, String>>> out) throws Exception {
+			int i = 0;
+			long l = 0;
+			long key = 0;
+
+			// collapse groups
+			for (Tuple2<Long, Tuple3<Integer, Long, String>> value : values) {
+				key = value.f0;
+				Tuple3<Integer, Long, String> extracted = value.f1;
+				i += extracted.f0;
+				l += extracted.f1;
+			}
+
+			Tuple3<Integer, Long, String> result = new Tuple3<>(i, l, "combined");
+			out.collect(new Tuple2<>(key, result));
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values,
+				Collector<Tuple2<Long, Tuple3<Integer, Long, String>>> out) throws Exception {
+			combine(values, out);
+		}
+	}
+
+	private static class Tuple3toTuple2GroupReduce implements KvGroupReduce<Long, Tuple3<Integer, Long, String>,
+			Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
+
+		@Override
+		public void combine(Iterable<Tuple2<Long, Tuple3<Integer, Long, String>>> values, Collector<Tuple2<Long,
+				Tuple2<Integer, Long>>> out) throws Exception {
+			int i = 0;
+			long l = 0;
+			long key = 0;
+
+			// collapse groups
+			for (Tuple2<Long, Tuple3<Integer, Long, String>> value : values) {
+				key = value.f0;
+				Tuple3<Integer, Long, String> extracted = value.f1;
+				i += extracted.f0;
+				l += extracted.f1 + extracted.f2.length();
+			}
+
+			Tuple2<Integer, Long> result = new Tuple2<>(i, l);
+			out.collect(new Tuple2<>(key, result));
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long,
+				Tuple2<Integer, Long>>> out) throws Exception {
+			new Tuple2toTuple2GroupReduce().reduce(values, out);
+		}
+	}
+
+	private static class Tuple2toTuple2GroupReduce implements KvGroupReduce<Long, Tuple2<Integer, Long>,
+			Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
+
+		@Override
+		public void combine(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long, Tuple2<Integer,
+				Long>>> out) throws Exception {
+			int i = 0;
+			long l = 0;
+			long key = 0;
+
+			// collapse groups
+			for (Tuple2<Long, Tuple2<Integer, Long>> value : values) {
+				key = value.f0;
+				Tuple2<Integer, Long> extracted = value.f1;
+				i += extracted.f0;
+				l += extracted.f1;
+			}
+
+			Tuple2<Integer, Long> result = new Tuple2<>(i, l);
+
+			out.collect(new Tuple2<>(key, result));
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple2<Long, Tuple2<Integer, Long>>> values, Collector<Tuple2<Long,
+				Tuple2<Integer, Long>>> out) throws Exception {
+			combine(values, out);
+		}
+	}
+
+	private class Tuple3KvWrapper implements MapFunction<Tuple3<Integer, Long, String>, Tuple2<Long,
+			Tuple3<Integer, Long, String>>> {
+		@Override
+		public Tuple2<Long, Tuple3<Integer, Long, String>> map(Tuple3<Integer, Long, String> value) throws Exception {
+			return new Tuple2<>(value.f1, value);
+		}
+	}
+
+	private interface CombineAndReduceGroup <IN, INT, OUT> extends GroupCombineFunction<IN, INT>,
+			GroupReduceFunction<INT, OUT> {
+	}
+
+	private interface KvGroupReduce<K, V, INT, OUT> extends CombineAndReduceGroup<Tuple2<K, V>, Tuple2<K, INT>,
+			Tuple2<K, OUT>> {
+	}
+
+}


[12/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
deleted file mode 100644
index 6efc565..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceITCase.java
+++ /dev/null
@@ -1,512 +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.javaApiOperators;
-
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.CustomType;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.PojoWithDateAndEnum;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class ReduceITCase extends MultipleProgramsTestBase {
-
-	public ReduceITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testReduceOnTuplesWithKeyFieldSelector() throws Exception {
-		/*
-		 * Reduce on tuples with key field selector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).reduce(new Tuple3Reduce("B-)"));
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"5,2,B-)\n" +
-				"15,3,B-)\n" +
-				"34,4,B-)\n" +
-				"65,5,B-)\n" +
-				"111,6,B-)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testReduceOnTupleWithMultipleKeyFieldSelectors() throws Exception{
-		/*
-		 * Reduce on tuples with multiple key field selectors
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
-				groupBy(4,0).reduce(new Tuple5Reduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
-				.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"3,9,0,P-),2\n" +
-				"3,6,5,BCD,3\n" +
-				"4,17,0,P-),1\n" +
-				"4,17,0,P-),2\n" +
-				"5,11,10,GHI,1\n" +
-				"5,29,0,P-),2\n" +
-				"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testReduceOnTuplesWithKeyExtractor() throws Exception {
-		/*
-		 * Reduce on tuples with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(new KeySelector1()).reduce(new Tuple3Reduce("B-)"));
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"5,2,B-)\n" +
-				"15,3,B-)\n" +
-				"34,4,B-)\n" +
-				"65,5,B-)\n" +
-				"111,6,B-)\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector1 implements KeySelector<Tuple3<Integer,Long,String>, Long> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Long getKey(Tuple3<Integer, Long, String> in) {
-			return in.f1;
-		}
-	}
-
-	@Test
-	public void testReduceOnCustomTypeWithKeyExtractor() throws Exception {
-		/*
-		 * Reduce on custom type with key extractor
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds.
-				groupBy(new KeySelector2()).reduce(new CustomTypeReduce());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "1,0,Hi\n" +
-				"2,3,Hello!\n" +
-				"3,12,Hello!\n" +
-				"4,30,Hello!\n" +
-				"5,60,Hello!\n" +
-				"6,105,Hello!\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class KeySelector2 implements KeySelector<CustomType, Integer> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Integer getKey(CustomType in) {
-			return in.myInt;
-		}
-	}
-
-	@Test
-	public void testAllReduceForTuple() throws Exception {
-		/*
-		 * All-reduce for tuple
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				reduce(new AllAddingTuple3Reduce());
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "231,91,Hello World\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testAllReduceForCustomTypes() throws Exception {
-		/*
-		 * All-reduce for custom types
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
-		DataSet<CustomType> reduceDs = ds.
-				reduce(new AllAddingCustomTypeReduce());
-
-		List<CustomType> result = reduceDs.collect();
-
-		String expected = "91,210,Hello!";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testReduceWithBroadcastSet() throws Exception {
-		/*
-		 * Reduce with broadcast set
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
-				groupBy(1).reduce(new BCTuple3Reduce()).withBroadcastSet(intDs, "ints");
-
-		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
-
-		String expected = "1,1,Hi\n" +
-				"5,2,55\n" +
-				"15,3,55\n" +
-				"34,4,55\n" +
-				"65,5,55\n" +
-				"111,6,55\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testReduceATupleReturningKeySelector() throws Exception {
-		/*
-		 * Reduce with a Tuple-returning KeySelector
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long,  Integer, String, Long>> reduceDs = ds .
-				groupBy(new KeySelector3()).reduce(new Tuple5Reduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
-				.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"3,9,0,P-),2\n" +
-				"3,6,5,BCD,3\n" +
-				"4,17,0,P-),1\n" +
-				"4,17,0,P-),2\n" +
-				"5,11,10,GHI,1\n" +
-				"5,29,0,P-),2\n" +
-				"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class KeySelector3 implements KeySelector<Tuple5<Integer,Long,Integer,String,Long>, Tuple2<Integer, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Long> getKey(Tuple5<Integer,Long,Integer,String,Long> t) {
-			return new Tuple2<Integer, Long>(t.f0, t.f4);
-		}
-	}
-
-	@Test
-	public void testReduceOnTupleWithMultipleKeyExpressions() throws Exception {
-		/*
-		 * Case 2 with String-based field expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
-				groupBy("f4","f0").reduce(new Tuple5Reduce());
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
-				.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-				"2,3,2,Hallo Welt wie,1\n" +
-				"2,2,1,Hallo Welt,2\n" +
-				"3,9,0,P-),2\n" +
-				"3,6,5,BCD,3\n" +
-				"4,17,0,P-),1\n" +
-				"4,17,0,P-),2\n" +
-				"5,11,10,GHI,1\n" +
-				"5,29,0,P-),2\n" +
-				"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testReduceOnTupleWithMultipleKeyExpressionsWithHashHint() throws Exception {
-		/*
-		 * Case 2 with String-based field expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
-			groupBy("f4","f0").reduce(new Tuple5Reduce()).setCombineHint(CombineHint.HASH);
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs
-			.collect();
-
-		String expected = "1,1,0,Hallo,1\n" +
-			"2,3,2,Hallo Welt wie,1\n" +
-			"2,2,1,Hallo Welt,2\n" +
-			"3,9,0,P-),2\n" +
-			"3,6,5,BCD,3\n" +
-			"4,17,0,P-),1\n" +
-			"4,17,0,P-),2\n" +
-			"5,11,10,GHI,1\n" +
-			"5,29,0,P-),2\n" +
-			"5,25,0,P-),3\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testSupportForDataAndEnumSerialization() throws Exception {
-		/**
-		 * Test support for Date and enum serialization
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		DataSet<PojoWithDateAndEnum> ds = env.generateSequence(0,2).map(new Mapper1());
-		ds = ds.union(CollectionDataSets.getPojoWithDateAndEnum(env));
-
-		DataSet<String> res = ds.groupBy("group").reduceGroup(new GroupReducer1());
-
-		List<String> result = res.collect();
-
-		String expected = "ok\nok";
-
-		compareResultAsText(result, expected);
-	}
-
-	public static class Mapper1 implements MapFunction<Long, PojoWithDateAndEnum> {
-		@Override
-		public PojoWithDateAndEnum map(Long value) throws Exception {
-			int l = value.intValue();
-			switch (l) {
-				case 0:
-					PojoWithDateAndEnum one = new PojoWithDateAndEnum();
-					one.group = "a";
-					one.date = new Date(666);
-					one.cat = CollectionDataSets.Category.CAT_A;
-					return one;
-				case 1:
-					PojoWithDateAndEnum two = new PojoWithDateAndEnum();
-					two.group = "a";
-					two.date = new Date(666);
-					two.cat = CollectionDataSets.Category.CAT_A;
-					return two;
-				case 2:
-					PojoWithDateAndEnum three = new PojoWithDateAndEnum();
-					three.group = "b";
-					three.date = new Date(666);
-					three.cat = CollectionDataSets.Category.CAT_B;
-					return three;
-			}
-			throw new RuntimeException("Unexpected value for l=" + l);
-		}
-	}
-
-	public static class GroupReducer1 implements GroupReduceFunction<CollectionDataSets.PojoWithDateAndEnum, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<PojoWithDateAndEnum> values,
-				Collector<String> out) throws Exception {
-			for(PojoWithDateAndEnum val : values) {
-				if(val.cat == CollectionDataSets.Category.CAT_A) {
-					Assert.assertEquals("a", val.group);
-				} else if(val.cat == CollectionDataSets.Category.CAT_B) {
-					Assert.assertEquals("b", val.group);
-				} else {
-					Assert.fail("error. Cat = "+val.cat);
-				}
-				Assert.assertEquals(666, val.date.getTime());
-			}
-			out.collect("ok");
-		}
-	}
-
-	public static class Tuple3Reduce implements ReduceFunction<Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
-		private final String f2Replace;
-
-		public Tuple3Reduce() {
-			this.f2Replace = null;
-		}
-
-		public Tuple3Reduce(String f2Replace) {
-			this.f2Replace = f2Replace;
-		}
-
-
-		@Override
-		public Tuple3<Integer, Long, String> reduce(
-				Tuple3<Integer, Long, String> in1,
-				Tuple3<Integer, Long, String> in2) throws Exception {
-
-			if(f2Replace == null) {
-				out.setFields(in1.f0+in2.f0, in1.f1, in1.f2);
-			} else {
-				out.setFields(in1.f0+in2.f0, in1.f1, this.f2Replace);
-			}
-			return out;
-		}
-	}
-
-	public static class Tuple5Reduce implements ReduceFunction<Tuple5<Integer, Long, Integer, String, Long>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple5<Integer, Long, Integer, String, Long> out = new Tuple5<Integer, Long, Integer, String, Long>();
-
-		@Override
-		public Tuple5<Integer, Long, Integer, String, Long> reduce(
-				Tuple5<Integer, Long, Integer, String, Long> in1,
-				Tuple5<Integer, Long, Integer, String, Long> in2)
-						throws Exception {
-
-			out.setFields(in1.f0, in1.f1+in2.f1, 0, "P-)", in1.f4);
-			return out;
-		}
-	}
-
-	public static class CustomTypeReduce implements ReduceFunction<CustomType> {
-		private static final long serialVersionUID = 1L;
-		private final CustomType out = new CustomType();
-
-		@Override
-		public CustomType reduce(CustomType in1, CustomType in2)
-				throws Exception {
-
-			out.myInt = in1.myInt;
-			out.myLong = in1.myLong + in2.myLong;
-			out.myString = "Hello!";
-			return out;
-		}
-	}
-
-	public static class AllAddingTuple3Reduce implements ReduceFunction<Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
-
-		@Override
-		public Tuple3<Integer, Long, String> reduce(
-				Tuple3<Integer, Long, String> in1,
-				Tuple3<Integer, Long, String> in2) throws Exception {
-
-			out.setFields(in1.f0+in2.f0, in1.f1+in2.f1, "Hello World");
-			return out;
-		}
-	}
-
-	public static class AllAddingCustomTypeReduce implements ReduceFunction<CustomType> {
-		private static final long serialVersionUID = 1L;
-		private final CustomType out = new CustomType();
-
-		@Override
-		public CustomType reduce(CustomType in1, CustomType in2)
-				throws Exception {
-
-			out.myInt = in1.myInt + in2.myInt;
-			out.myLong = in1.myLong + in2.myLong;
-			out.myString = "Hello!";
-			return out;
-		}
-	}
-
-	public static class BCTuple3Reduce extends RichReduceFunction<Tuple3<Integer, Long, String>> {
-		private static final long serialVersionUID = 1L;
-		private final Tuple3<Integer, Long, String> out = new Tuple3<Integer, Long, String>();
-		private String f2Replace = "";
-
-		@Override
-		public void open(Configuration config) {
-
-			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
-			int sum = 0;
-			for(Integer i : ints) {
-				sum += i;
-			}
-			f2Replace = sum+"";
-
-		}
-
-		@Override
-		public Tuple3<Integer, Long, String> reduce(
-				Tuple3<Integer, Long, String> in1,
-				Tuple3<Integer, Long, String> in2) throws Exception {
-
-			out.setFields(in1.f0+in2.f0, in1.f1, this.f2Replace);
-			return out;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceWithCombinerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceWithCombinerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceWithCombinerITCase.java
deleted file mode 100644
index 685a9ac..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReduceWithCombinerITCase.java
+++ /dev/null
@@ -1,313 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.CombineFunction;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.UnsortedGrouping;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Arrays;
-import java.util.List;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class ReduceWithCombinerITCase extends MultipleProgramsTestBase {
-
-	public ReduceWithCombinerITCase(TestExecutionMode mode) {
-		super(TestExecutionMode.CLUSTER);
-	}
-
-	@Test
-	public void testReduceOnNonKeyedDataset() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple2<Integer, Boolean>> input = createNonKeyedInput(env);
-		List<Tuple2<Integer, Boolean>> actual = input.reduceGroup(new NonKeyedCombReducer()).collect();
-		String expected = "10,true\n";
-
-		compareResultAsTuples(actual, expected);
-	}
-
-	@Test
-	public void testForkingReduceOnNonKeyedDataset() throws Exception {
-
-		// set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple2<Integer, Boolean>> input = createNonKeyedInput(env);
-
-		DataSet<Tuple2<Integer, Boolean>> r1 = input.reduceGroup(new NonKeyedCombReducer());
-		DataSet<Tuple2<Integer, Boolean>> r2 = input.reduceGroup(new NonKeyedGroupCombReducer());
-
-		List<Tuple2<Integer, Boolean>> actual = r1.union(r2).collect();
-		String expected = "10,true\n10,true\n";
-		compareResultAsTuples(actual, expected);
-	}
-
-	@Test
-	public void testReduceOnKeyedDataset() throws Exception {
-
-		// set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
-		List<Tuple3<String, Integer, Boolean>> actual = input.groupBy(0).reduceGroup(new KeyedCombReducer()).collect();
-		String expected = "k1,6,true\nk2,4,true\n";
-
-		compareResultAsTuples(actual, expected);
-	}
-
-	@Test
-	public void testReduceOnKeyedDatasetWithSelector() throws Exception {
-
-		// set up the execution environment
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
-
-		List<Tuple3<String, Integer, Boolean>> actual = input
-			.groupBy(new KeySelectorX())
-			.reduceGroup(new KeyedCombReducer())
-			.collect();
-		String expected = "k1,6,true\nk2,4,true\n";
-
-		compareResultAsTuples(actual, expected);
-	}
-
-	@Test
-	public void testForkingReduceOnKeyedDataset() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
-
-		UnsortedGrouping<Tuple3<String, Integer, Boolean>> counts = input.groupBy(0);
-
-		DataSet<Tuple3<String, Integer, Boolean>> r1 = counts.reduceGroup(new KeyedCombReducer());
-		DataSet<Tuple3<String, Integer, Boolean>> r2 = counts.reduceGroup(new KeyedGroupCombReducer());
-
-		List<Tuple3<String, Integer, Boolean>> actual = r1.union(r2).collect();
-		String expected = "k1,6,true\n" +
-			"k2,4,true\n" +
-			"k1,6,true\n" +
-			"k2,4,true\n";
-		compareResultAsTuples(actual, expected);
-	}
-
-	@Test
-	public void testForkingReduceOnKeyedDatasetWithSelection() throws Exception {
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		// creates the input data and distributes them evenly among the available downstream tasks
-		DataSet<Tuple3<String, Integer, Boolean>> input = createKeyedInput(env);
-
-		UnsortedGrouping<Tuple3<String, Integer, Boolean>> counts = input.groupBy(new KeySelectorX());
-
-		DataSet<Tuple3<String, Integer, Boolean>> r1 = counts.reduceGroup(new KeyedCombReducer());
-		DataSet<Tuple3<String, Integer, Boolean>> r2 = counts.reduceGroup(new KeyedGroupCombReducer());
-
-		List<Tuple3<String, Integer, Boolean>> actual = r1.union(r2).collect();
-		String expected = "k1,6,true\n" +
-			"k2,4,true\n" +
-			"k1,6,true\n" +
-			"k2,4,true\n";
-
-		compareResultAsTuples(actual, expected);
-	}
-
-	private DataSet<Tuple2<Integer, Boolean>> createNonKeyedInput(ExecutionEnvironment env) {
-		return env.fromCollection(Arrays.asList(
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false),
-			new Tuple2<>(1, false))
-		).rebalance();
-	}
-
-	private static class NonKeyedCombReducer implements CombineFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>>,
-		GroupReduceFunction<Tuple2<Integer, Boolean>,Tuple2<Integer, Boolean>> {
-
-		@Override
-		public Tuple2<Integer, Boolean> combine(Iterable<Tuple2<Integer, Boolean>> values) throws Exception {
-			int sum = 0;
-			boolean flag = true;
-
-			for(Tuple2<Integer, Boolean> tuple : values) {
-				sum += tuple.f0;
-				flag &= !tuple.f1;
-
-			}
-			return new Tuple2<>(sum, flag);
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
-			int sum = 0;
-			boolean flag = true;
-			for(Tuple2<Integer, Boolean> tuple : values) {
-				sum += tuple.f0;
-				flag &= tuple.f1;
-			}
-			out.collect(new Tuple2<>(sum, flag));
-		}
-	}
-
-	private static class NonKeyedGroupCombReducer implements GroupCombineFunction<Tuple2<Integer, Boolean>, Tuple2<Integer, Boolean>>,
-		GroupReduceFunction<Tuple2<Integer, Boolean>,Tuple2<Integer, Boolean>> {
-
-		@Override
-		public void reduce(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
-			int sum = 0;
-			boolean flag = true;
-			for(Tuple2<Integer, Boolean> tuple : values) {
-				sum += tuple.f0;
-				flag &= tuple.f1;
-			}
-			out.collect(new Tuple2<>(sum, flag));
-		}
-
-		@Override
-		public void combine(Iterable<Tuple2<Integer, Boolean>> values, Collector<Tuple2<Integer, Boolean>> out) throws Exception {
-			int sum = 0;
-			boolean flag = true;
-			for(Tuple2<Integer, Boolean> tuple : values) {
-				sum += tuple.f0;
-				flag &= !tuple.f1;
-			}
-			out.collect(new Tuple2<>(sum, flag));
-		}
-	}
-
-	private DataSet<Tuple3<String, Integer, Boolean>> createKeyedInput(ExecutionEnvironment env) {
-		return env.fromCollection(Arrays.asList(
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k2", 1, false),
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k2", 1, false),
-			new Tuple3<>("k2", 1, false),
-			new Tuple3<>("k1", 1, false),
-			new Tuple3<>("k2", 1, false))
-		).rebalance();
-	}
-
-	public static class KeySelectorX implements KeySelector<Tuple3<String, Integer, Boolean>, String> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public String getKey(Tuple3<String, Integer, Boolean> in) {
-			return in.f0;
-		}
-	}
-
-	private class KeyedCombReducer implements CombineFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>>,
-		GroupReduceFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>> {
-
-		@Override
-		public Tuple3<String, Integer, Boolean> combine(Iterable<Tuple3<String, Integer, Boolean>> values) throws Exception {
-			String key = null;
-			int sum = 0;
-			boolean flag = true;
-			
-			for(Tuple3<String, Integer, Boolean> tuple : values) {
-				key = (key == null) ? tuple.f0 : key;
-				sum += tuple.f1;
-				flag &= !tuple.f2;
-			}
-			return new Tuple3<>(key, sum, flag);
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
-			String key = null;
-			int sum = 0;
-			boolean flag = true;
-
-			for(Tuple3<String, Integer, Boolean> tuple : values) {
-				key = (key == null) ? tuple.f0 : key;
-				sum += tuple.f1;
-				flag &= tuple.f2;
-			}
-			out.collect(new Tuple3<>(key, sum, flag));
-		}
-	}
-
-	private class KeyedGroupCombReducer implements GroupCombineFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>>,
-		GroupReduceFunction<Tuple3<String, Integer, Boolean>, Tuple3<String, Integer, Boolean>> {
-
-		@Override
-		public void combine(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
-			String key = null;
-			int sum = 0;
-			boolean flag = true;
-
-			for(Tuple3<String, Integer, Boolean> tuple : values) {
-				key = (key == null) ? tuple.f0 : key;
-				sum += tuple.f1;
-				flag &= !tuple.f2;
-			}
-			out.collect(new Tuple3<>(key, sum, flag));
-		}
-
-		@Override
-		public void reduce(Iterable<Tuple3<String, Integer, Boolean>> values, Collector<Tuple3<String, Integer, Boolean>> out) throws Exception {
-			String key = null;
-			int sum = 0;
-			boolean flag = true;
-
-			for(Tuple3<String, Integer, Boolean> tuple : values) {
-				key = (key == null) ? tuple.f0 : key;
-				sum += tuple.f1;
-				flag &= tuple.f2;
-			}
-			out.collect(new Tuple3<>(key, sum, flag));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
deleted file mode 100644
index 85961db..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.api.common.io.GenericInputFormat;
-import org.apache.flink.api.common.operators.util.TestNonRichInputFormat;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.AkkaOptions;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.runtime.minicluster.StandaloneMiniCluster;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-@SuppressWarnings("serial")
-public class RemoteEnvironmentITCase extends TestLogger {
-
-	private static final int TM_SLOTS = 4;
-
-	private static final int USER_DOP = 2;
-
-	private static final String INVALID_STARTUP_TIMEOUT = "0.001 ms";
-
-	private static final String VALID_STARTUP_TIMEOUT = "100 s";
-
-	private static Configuration configuration;
-
-	private static StandaloneMiniCluster cluster;
-
-
-	@BeforeClass
-	public static void setupCluster() throws Exception {
-		configuration = new Configuration();
-
-		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
-
-		cluster = new StandaloneMiniCluster(configuration);
-	}
-
-	@AfterClass
-	public static void tearDownCluster() throws Exception {
-		cluster.close();
-	}
-
-	/**
-	 * Ensure that that Akka configuration parameters can be set.
-	 */
-	@Test(expected=FlinkException.class)
-	public void testInvalidAkkaConfiguration() throws Throwable {
-		Configuration config = new Configuration();
-		config.setString(AkkaOptions.STARTUP_TIMEOUT, INVALID_STARTUP_TIMEOUT);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				cluster.getHostname(),
-				cluster.getPort(),
-				config
-		);
-		env.getConfig().disableSysoutLogging();
-
-		DataSet<String> result = env.createInput(new TestNonRichInputFormat());
-		result.output(new LocalCollectionOutputFormat<>(new ArrayList<String>()));
-		try {
-			env.execute();
-			Assert.fail("Program should not run successfully, cause of invalid akka settings.");
-		} catch (ProgramInvocationException ex) {
-			throw ex.getCause();
-		}
-	}
-
-	/**
-	 * Ensure that the program parallelism can be set even if the configuration is supplied.
-	 */
-	@Test
-	public void testUserSpecificParallelism() throws Exception {
-		Configuration config = new Configuration();
-		config.setString(AkkaOptions.STARTUP_TIMEOUT, VALID_STARTUP_TIMEOUT);
-
-		final ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
-				cluster.getHostname(),
-				cluster.getPort(),
-				config
-		);
-		env.setParallelism(USER_DOP);
-		env.getConfig().disableSysoutLogging();
-
-		DataSet<Integer> result = env.createInput(new ParallelismDependentInputFormat())
-				.rebalance()
-				.mapPartition(new RichMapPartitionFunction<Integer, Integer>() {
-					@Override
-					public void mapPartition(Iterable<Integer> values, Collector<Integer> out) throws Exception {
-						out.collect(getRuntimeContext().getIndexOfThisSubtask());
-					}
-				});
-		List<Integer> resultCollection = result.collect();
-		assertEquals(USER_DOP, resultCollection.size());
-	}
-
-	private static class ParallelismDependentInputFormat extends GenericInputFormat<Integer> {
-
-		private transient boolean emitted;
-
-		@Override
-		public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
-			assertEquals(USER_DOP, numSplits);
-			return super.createInputSplits(numSplits);
-		}
-
-		@Override
-		public boolean reachedEnd() {
-			return emitted;
-		}
-
-		@Override
-		public Integer nextRecord(Integer reuse) {
-			if (emitted) {
-				return null;
-			}
-			emitted = true;
-			return 1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
deleted file mode 100644
index 8cc54aa..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ReplicatingDataSourceITCase.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.javaApiOperators;
-
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.io.ReplicatingInputFormat;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.ParallelIteratorInputFormat;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.NumberSequenceIterator;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-/**
- * Tests for replicating DataSources
- */
-
-@RunWith(Parameterized.class)
-public class ReplicatingDataSourceITCase extends MultipleProgramsTestBase {
-
-	public ReplicatingDataSourceITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testReplicatedSourceToJoin() throws Exception {
-		/*
-		 * Test replicated source going into join
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple1<Long>> source1 = env.createInput(new ReplicatingInputFormat<Long, GenericInputSplit>
-				(new ParallelIteratorInputFormat<Long>(new NumberSequenceIterator(0l, 1000l))), BasicTypeInfo.LONG_TYPE_INFO)
-				.map(new ToTuple());
-		DataSet<Tuple1<Long>> source2 = env.generateSequence(0l, 1000l).map(new ToTuple());
-
-		DataSet<Tuple> pairs = source1.join(source2).where(0).equalTo(0)
-				.projectFirst(0)
-				.sum(0);
-
-		List<Tuple> result = pairs.collect();
-
-		String expectedResult = "(500500)";
-
-		compareResultAsText(result, expectedResult);
-	}
-
-	@Test
-	public void testReplicatedSourceToCross() throws Exception {
-		/*
-		 * Test replicated source going into cross
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple1<Long>> source1 = env.createInput(new ReplicatingInputFormat<Long, GenericInputSplit>
-				(new ParallelIteratorInputFormat<Long>(new NumberSequenceIterator(0l, 1000l))), BasicTypeInfo.LONG_TYPE_INFO)
-				.map(new ToTuple());
-		DataSet<Tuple1<Long>> source2 = env.generateSequence(0l, 1000l).map(new ToTuple());
-
-		DataSet<Tuple1<Long>> pairs = source1.cross(source2)
-				.filter(new FilterFunction<Tuple2<Tuple1<Long>, Tuple1<Long>>>() {
-					@Override
-					public boolean filter(Tuple2<Tuple1<Long>, Tuple1<Long>> value) throws Exception {
-						return value.f0.f0.equals(value.f1.f0);
-					}
-				})
-				.map(new MapFunction<Tuple2<Tuple1<Long>, Tuple1<Long>>, Tuple1<Long>>() {
-					@Override
-					public Tuple1<Long> map(Tuple2<Tuple1<Long>, Tuple1<Long>> value) throws Exception {
-						return value.f0;
-					}
-				})
-				.sum(0);
-
-		List<Tuple1<Long>> result = pairs.collect();
-
-		String expectedResult = "(500500)";
-
-		compareResultAsText(result, expectedResult);
-	}
-
-
-	public static class ToTuple implements MapFunction<Long, Tuple1<Long>> {
-
-		@Override
-		public Tuple1<Long> map(Long value) throws Exception {
-			return new Tuple1<Long>(value);
-		}
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java
deleted file mode 100644
index a9c75e5..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SampleITCase.java
+++ /dev/null
@@ -1,167 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.FlatMapOperator;
-import org.apache.flink.api.java.operators.MapPartitionOperator;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-import java.util.Random;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-@SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class SampleITCase extends MultipleProgramsTestBase {
-
-	private static final Random RNG = new Random();
-
-	public SampleITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Before
-	public void initiate() {
-		ExecutionEnvironment.getExecutionEnvironment().setParallelism(5);
-	}
-
-	@Test
-	public void testSamplerWithFractionWithoutReplacement() throws Exception {
-		verifySamplerWithFractionWithoutReplacement(0d);
-		verifySamplerWithFractionWithoutReplacement(0.2d);
-		verifySamplerWithFractionWithoutReplacement(1.0d);
-	}
-
-	@Test
-	public void testSamplerWithFractionWithReplacement() throws Exception {
-		verifySamplerWithFractionWithReplacement(0d);
-		verifySamplerWithFractionWithReplacement(0.2d);
-		verifySamplerWithFractionWithReplacement(1.0d);
-		verifySamplerWithFractionWithReplacement(2.0d);
-	}
-
-	@Test
-	public void testSamplerWithSizeWithoutReplacement() throws Exception {
-		verifySamplerWithFixedSizeWithoutReplacement(0);
-		verifySamplerWithFixedSizeWithoutReplacement(2);
-		verifySamplerWithFixedSizeWithoutReplacement(21);
-	}
-
-	@Test
-	public void testSamplerWithSizeWithReplacement() throws Exception {
-		verifySamplerWithFixedSizeWithReplacement(0);
-		verifySamplerWithFixedSizeWithReplacement(2);
-		verifySamplerWithFixedSizeWithReplacement(21);
-	}
-
-	private void verifySamplerWithFractionWithoutReplacement(double fraction) throws Exception {
-		verifySamplerWithFractionWithoutReplacement(fraction, RNG.nextLong());
-	}
-
-	private void verifySamplerWithFractionWithoutReplacement(double fraction, long seed) throws Exception {
-		verifySamplerWithFraction(false, fraction, seed);
-	}
-
-	private void verifySamplerWithFractionWithReplacement(double fraction) throws Exception {
-		verifySamplerWithFractionWithReplacement(fraction, RNG.nextLong());
-	}
-
-	private void verifySamplerWithFractionWithReplacement(double fraction, long seed) throws Exception {
-		verifySamplerWithFraction(true, fraction, seed);
-	}
-
-	private void verifySamplerWithFraction(boolean withReplacement, double fraction, long seed) throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		FlatMapOperator<Tuple3<Integer, Long, String>, String> ds = getSourceDataSet(env);
-		MapPartitionOperator<String, String> sampled = DataSetUtils.sample(ds, withReplacement, fraction, seed);
-		List<String> result = sampled.collect();
-		containsResultAsText(result, getSourceStrings());
-	}
-
-	private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples) throws Exception {
-		verifySamplerWithFixedSizeWithoutReplacement(numSamples, RNG.nextLong());
-	}
-
-	private void verifySamplerWithFixedSizeWithoutReplacement(int numSamples, long seed) throws Exception {
-		verifySamplerWithFixedSize(false, numSamples, seed);
-	}
-
-	private void verifySamplerWithFixedSizeWithReplacement(int numSamples) throws Exception {
-		verifySamplerWithFixedSizeWithReplacement(numSamples, RNG.nextLong());
-	}
-
-	private void verifySamplerWithFixedSizeWithReplacement(int numSamples, long seed) throws Exception {
-		verifySamplerWithFixedSize(true, numSamples, seed);
-	}
-
-	private void verifySamplerWithFixedSize(boolean withReplacement, int numSamples, long seed) throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		FlatMapOperator<Tuple3<Integer, Long, String>, String> ds = getSourceDataSet(env);
-		DataSet<String> sampled = DataSetUtils.sampleWithSize(ds, withReplacement, numSamples, seed);
-		List<String> result = sampled.collect();
-		assertEquals(numSamples, result.size());
-		containsResultAsText(result, getSourceStrings());
-	}
-
-	private FlatMapOperator<Tuple3<Integer, Long, String>, String> getSourceDataSet(ExecutionEnvironment env) {
-		return CollectionDataSets.get3TupleDataSet(env).flatMap(
-			new FlatMapFunction<Tuple3<Integer, Long, String>, String>() {
-				@Override
-				public void flatMap(Tuple3<Integer, Long, String> value, Collector<String> out) throws Exception {
-					out.collect(value.f2);
-				}
-			});
-	}
-	
-	private String getSourceStrings() {
-		return "Hi\n" +
-			"Hello\n" +
-			"Hello world\n" +
-			"Hello world, how are you?\n" +
-			"I am fine.\n" +
-			"Luke Skywalker\n" +
-			"Comment#1\n" +
-			"Comment#2\n" +
-			"Comment#3\n" +
-			"Comment#4\n" +
-			"Comment#5\n" +
-			"Comment#6\n" +
-			"Comment#7\n" +
-			"Comment#8\n" +
-			"Comment#9\n" +
-			"Comment#10\n" +
-			"Comment#11\n" +
-			"Comment#12\n" +
-			"Comment#13\n" +
-			"Comment#14\n" +
-			"Comment#15\n";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
deleted file mode 100644
index c7f07f6..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SortPartitionITCase.java
+++ /dev/null
@@ -1,343 +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.javaApiOperators;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.MapPartitionFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets.POJO;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class SortPartitionITCase extends MultipleProgramsTestBase {
-
-	public SortPartitionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testSortPartitionByKeyField() throws Exception {
-		/*
-		 * Test sort partition on key field
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
-				.sortPartition(1, Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionByTwoKeyFields() throws Exception {
-		/*
-		 * Test sort partition on two key fields
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(2);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper<Tuple5<Integer, Long, Integer, String, Long>>()).setParallelism(2) // parallelize input
-				.sortPartition(4, Order.ASCENDING)
-				.sortPartition(2, Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new Tuple5Checker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testSortPartitionByFieldExpression() throws Exception {
-		/*
-		 * Test sort partition on field expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper()).setParallelism(4) // parallelize input
-				.sortPartition("f1", Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionByTwoFieldExpressions() throws Exception {
-		/*
-		 * Test sort partition on two field expressions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(2);
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper<Tuple5<Integer, Long, Integer, String, Long>>()).setParallelism(2) // parallelize input
-				.sortPartition("f4", Order.ASCENDING)
-				.sortPartition("f2", Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new Tuple5Checker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionByNestedFieldExpression() throws Exception {
-		/*
-		 * Test sort partition on nested field expressions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper<Tuple2<Tuple2<Integer, Integer>, String>>()).setParallelism(3) // parallelize input
-				.sortPartition("f0.f1", Order.ASCENDING)
-				.sortPartition("f1", Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new NestedTupleChecker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionPojoByNestedFieldExpression() throws Exception {
-		/*
-		 * Test sort partition on field expression
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<POJO> ds = CollectionDataSets.getMixedPojoDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.map(new IdMapper<POJO>()).setParallelism(1) // parallelize input
-				.sortPartition("nestedTupleWithCustom.f1.myString", Order.ASCENDING)
-				.sortPartition("number", Order.DESCENDING)
-				.mapPartition(new OrderCheckMapper<>(new PojoChecker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionParallelismChange() throws Exception {
-		/*
-		 * Test sort partition with parallelism change
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-				.sortPartition(1, Order.DESCENDING).setParallelism(3) // change parallelism
-				.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
-				.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionWithKeySelector1() throws Exception {
-		/*
-		 * Test sort partition on an extracted key
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-			.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
-			.sortPartition(new KeySelector<Tuple3<Integer, Long, String>, Long>() {
-				@Override
-				public Long getKey(Tuple3<Integer, Long, String> value) throws Exception {
-					return value.f1;
-				}
-			}, Order.ASCENDING)
-			.mapPartition(new OrderCheckMapper<>(new Tuple3AscendingChecker()))
-			.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testSortPartitionWithKeySelector2() throws Exception {
-		/*
-		 * Test sort partition on an extracted key
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		List<Tuple1<Boolean>> result = ds
-			.map(new IdMapper<Tuple3<Integer, Long, String>>()).setParallelism(4) // parallelize input
-			.sortPartition(new KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>>() {
-				@Override
-				public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> value) throws Exception {
-					return new Tuple2<>(value.f0, value.f1);
-				}
-			}, Order.DESCENDING)
-			.mapPartition(new OrderCheckMapper<>(new Tuple3Checker()))
-			.distinct().collect();
-
-		String expected = "(true)\n";
-
-		compareResultAsText(result, expected);
-	}
-
-	public interface OrderChecker<T> extends Serializable {
-		boolean inOrder(T t1, T t2);
-	}
-
-	@SuppressWarnings("serial")
-	public static class Tuple3Checker implements OrderChecker<Tuple3<Integer, Long, String>> {
-		@Override
-		public boolean inOrder(Tuple3<Integer, Long, String> t1, Tuple3<Integer, Long, String> t2) {
-			return t1.f1 >= t2.f1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class Tuple3AscendingChecker implements OrderChecker<Tuple3<Integer, Long, String>> {
-		@Override
-		public boolean inOrder(Tuple3<Integer, Long, String> t1, Tuple3<Integer, Long, String> t2) {
-			return t1.f1 <= t2.f1;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class Tuple5Checker implements OrderChecker<Tuple5<Integer, Long, Integer, String, Long>> {
-		@Override
-		public boolean inOrder(Tuple5<Integer, Long, Integer, String, Long> t1,
-				Tuple5<Integer, Long, Integer, String, Long> t2) {
-			return t1.f4 < t2.f4 || t1.f4.equals(t2.f4) && t1.f2 >= t2.f2;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class NestedTupleChecker implements OrderChecker<Tuple2<Tuple2<Integer, Integer>, String>> {
-		@Override
-		public boolean inOrder(Tuple2<Tuple2<Integer, Integer>, String> t1,
-				Tuple2<Tuple2<Integer, Integer>, String> t2) {
-			return t1.f0.f1 < t2.f0.f1 ||
-					t1.f0.f1.equals(t2.f0.f1) && t1.f1.compareTo(t2.f1) >= 0;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class PojoChecker implements OrderChecker<POJO> {
-		@Override
-		public boolean inOrder(POJO t1, POJO t2) {
-			return t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) < 0 ||
-					t1.nestedTupleWithCustom.f1.myString.compareTo(t2.nestedTupleWithCustom.f1.myString) == 0 &&
-					t1.number >= t2.number;
-		}
-	}
-
-	@SuppressWarnings("unused, serial")
-	public static class OrderCheckMapper<T> implements MapPartitionFunction<T, Tuple1<Boolean>> {
-
-		OrderChecker<T> checker;
-
-		public OrderCheckMapper() {}
-
-		public OrderCheckMapper(OrderChecker<T> checker) {
-			this.checker = checker;
-		}
-
-		@Override
-		public void mapPartition(Iterable<T> values, Collector<Tuple1<Boolean>> out) throws Exception {
-
-			Iterator<T> it = values.iterator();
-			if(!it.hasNext()) {
-				out.collect(new Tuple1<>(true));
-			} else {
-				T last = it.next();
-
-				while (it.hasNext()) {
-					T next = it.next();
-					if (!checker.inOrder(last, next)) {
-						out.collect(new Tuple1<>(false));
-						return;
-					}
-					last = next;
-				}
-				out.collect(new Tuple1<>(true));
-			}
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class IdMapper<T> implements MapFunction<T, T> {
-
-		@Override
-		public T map(T value) throws Exception {
-			return value;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
deleted file mode 100644
index e5bdc19..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/SumMinMaxITCase.java
+++ /dev/null
@@ -1,103 +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.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class SumMinMaxITCase extends MultipleProgramsTestBase {
-
-	public SumMinMaxITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testSumMaxAndProject() throws Exception {
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Integer, Long>> sumDs = ds
-				.sum(0)
-				.andMax(1)
-				.project(0, 1);
-
-		List<Tuple2<Integer, Long>> result = sumDs.collect();
-
-		String expected = "231,6\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testGroupedAggregate() throws Exception {
-		/*
-		 * Grouped Aggregate
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple2<Long, Integer>> aggregateDs = ds.groupBy(1)
-				.sum(0)
-				.project(1, 0);
-
-		List<Tuple2<Long, Integer>> result = aggregateDs.collect();
-
-		String expected = "1,1\n" +
-				"2,5\n" +
-				"3,15\n" +
-				"4,34\n" +
-				"5,65\n" +
-				"6,111\n";
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testNestedAggregate() throws Exception {
-		/*
-		 * Nested Aggregate
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple1<Integer>> aggregateDs = ds.groupBy(1)
-				.min(0)
-				.min(0)
-				.project(0);
-
-		List<Tuple1<Integer>> result = aggregateDs.collect();
-
-		String expected = "1\n";
-
-		compareResultAsTuples(result, expected);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java
deleted file mode 100644
index 067939f..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/TypeHintITCase.java
+++ /dev/null
@@ -1,326 +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.javaApiOperators;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.JavaProgramTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class TypeHintITCase extends JavaProgramTestBase {
-
-	private static int NUM_PROGRAMS = 9;
-
-	private int curProgId = config.getInteger("ProgramId", -1);
-
-	public TypeHintITCase(Configuration config) {
-		super(config);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		TypeHintProgs.runProgram(curProgId);
-	}
-
-	@Parameters
-	public static Collection<Object[]> getConfigurations() throws FileNotFoundException, IOException {
-
-		LinkedList<Configuration> tConfigs = new LinkedList<Configuration>();
-
-		for(int i=1; i <= NUM_PROGRAMS; i++) {
-			Configuration config = new Configuration();
-			config.setInteger("ProgramId", i);
-			tConfigs.add(config);
-		}
-
-		return toParameterList(tConfigs);
-	}
-
-	private static class TypeHintProgs {
-
-		public static void runProgram(int progId) throws Exception {
-			switch(progId) {
-			// Test identity map with missing types and string type hint
-			case 1: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds
-						.map(new Mapper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>())
-						.returns("Tuple3<Integer, Long, String>");
-				List<Tuple3<Integer, Long, String>> result = identityMapDs.collect();
-
-				String expectedResult = "(2,2,Hello)\n" +
-						"(3,2,Hello world)\n" +
-						"(1,1,Hi)\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test identity map with missing types and type information type hint
-			case 2: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Tuple3<Integer, Long, String>> identityMapDs = ds
-						// all following generics get erased during compilation
-						.map(new Mapper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>())
-						.returns(new TupleTypeInfo<Tuple3<Integer, Long, String>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO));
-				List<Tuple3<Integer, Long, String>> result = identityMapDs
-						.collect();
-
-				String expectedResult = "(2,2,Hello)\n" +
-						"(3,2,Hello world)\n" +
-						"(1,1,Hi)\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test flat map with class type hint
-			case 3: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> identityMapDs = ds
-						.flatMap(new FlatMapper<Tuple3<Integer, Long, String>, Integer>())
-						.returns(Integer.class);
-				List<Integer> result = identityMapDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test join with type information type hint
-			case 4: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds1
-						.join(ds2)
-						.where(0)
-						.equalTo(0)
-						.with(new Joiner<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test flat join with type information type hint
-			case 5: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds1
-						.join(ds2)
-						.where(0)
-						.equalTo(0)
-						.with(new FlatJoiner<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test unsorted group reduce with type information type hint
-			case 6: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds
-						.groupBy(0)
-						.reduceGroup(new GroupReducer<Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test sorted group reduce with type information type hint
-			case 7: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds
-						.groupBy(0)
-						.sortGroup(0, Order.ASCENDING)
-						.reduceGroup(new GroupReducer<Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test combine group with type information type hint
-			case 8: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds
-						.groupBy(0)
-						.combineGroup(new GroupCombiner<Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			// Test cogroup with type information type hint
-			case 9: {
-				final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-				DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
-				DataSet<Integer> resultDs = ds1
-						.coGroup(ds2)
-						.where(0)
-						.equalTo(0)
-						.with(new CoGrouper<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>, Integer>())
-						.returns(BasicTypeInfo.INT_TYPE_INFO);
-				List<Integer> result = resultDs.collect();
-
-				String expectedResult = "2\n" +
-						"3\n" +
-						"1\n";
-
-				compareResultAsText(result, expectedResult);
-				break;
-			}
-			default:
-				throw new IllegalArgumentException("Invalid program id");
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	public static class Mapper<T, V> implements MapFunction<T, V> {
-		private static final long serialVersionUID = 1L;
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public V map(T value) throws Exception {
-			return (V) value;
-		}
-	}
-
-	public static class FlatMapper<T, V> implements FlatMapFunction<T, V> {
-		private static final long serialVersionUID = 1L;
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		@Override
-		public void flatMap(T value, Collector<V> out) throws Exception {
-			out.collect((V) ((Tuple3)value).f0);
-		}
-	}
-
-	public static class Joiner<IN1, IN2, OUT> implements JoinFunction<IN1, IN2, OUT> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public OUT join(IN1 first, IN2 second) throws Exception {
-			return (OUT) ((Tuple3) first).f0;
-		}
-	}
-
-	public static class FlatJoiner<IN1, IN2, OUT> implements FlatJoinFunction<IN1, IN2, OUT> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void join(IN1 first, IN2 second, Collector<OUT> out) throws Exception {
-			out.collect((OUT) ((Tuple3) first).f0);
-		}
-	}
-
-	public static class GroupReducer<IN, OUT> implements GroupReduceFunction<IN, OUT> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterable<IN> values, Collector<OUT> out) throws Exception {
-			out.collect((OUT) ((Tuple3) values.iterator().next()).f0);
-		}
-	}
-
-	public static class GroupCombiner<IN, OUT> implements GroupCombineFunction<IN, OUT> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void combine(Iterable<IN> values, Collector<OUT> out) throws Exception {
-			out.collect((OUT) ((Tuple3) values.iterator().next()).f0);
-		}
-	}
-
-	public static class CoGrouper<IN1, IN2, OUT> implements CoGroupFunction<IN1, IN2, OUT> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void coGroup(Iterable<IN1> first, Iterable<IN2> second, Collector<OUT> out) throws Exception {
-			out.collect((OUT) ((Tuple3) first.iterator().next()).f0);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
deleted file mode 100644
index 7ab2764..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/UnionITCase.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.javaApiOperators;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-@RunWith(Parameterized.class)
-public class UnionITCase extends MultipleProgramsTestBase {
-
-	private static final String FULL_TUPLE_3_STRING = "1,1,Hi\n" +
-			"2,2,Hello\n" +
-			"3,2,Hello world\n" +
-			"4,3,Hello world, how are you?\n" +
-			"5,3,I am fine.\n" +
-			"6,3,Luke Skywalker\n" +
-			"7,4,Comment#1\n" +
-			"8,4,Comment#2\n" +
-			"9,4,Comment#3\n" +
-			"10,4,Comment#4\n" +
-			"11,5,Comment#5\n" +
-			"12,5,Comment#6\n" +
-			"13,5,Comment#7\n" +
-			"14,5,Comment#8\n" +
-			"15,5,Comment#9\n" +
-			"16,6,Comment#10\n" +
-			"17,6,Comment#11\n" +
-			"18,6,Comment#12\n" +
-			"19,6,Comment#13\n" +
-			"20,6,Comment#14\n" +
-			"21,6,Comment#15\n";
-
-	public UnionITCase(TestExecutionMode mode){
-		super(mode);
-	}
-
-	@Test
-	public void testUnion2IdenticalDataSets() throws Exception {
-		/*
-		 * Union of 2 Same Data Sets
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env));
-
-		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
-
-		String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING;
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testUnion5IdenticalDataSets() throws Exception {
-		/*
-		 * Union of 5 same Data Sets, with multiple unions
-		 */
-
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple3<Integer, Long, String>> unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env))
-				.union(CollectionDataSets.get3TupleDataSet(env))
-				.union(CollectionDataSets.get3TupleDataSet(env))
-				.union(CollectionDataSets.get3TupleDataSet(env));
-
-		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
-
-		String expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING
-				+ FULL_TUPLE_3_STRING +
-				FULL_TUPLE_3_STRING +	FULL_TUPLE_3_STRING;
-
-		compareResultAsTuples(result, expected);
-	}
-
-	@Test
-	public void testUnionWithEmptyDataSet() throws Exception {
-		/*
-		 * Test on union with empty dataset
-		 */
-		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		// Don't know how to make an empty result in an other way than filtering it
-		DataSet<Tuple3<Integer, Long, String>> empty = CollectionDataSets.get3TupleDataSet(env).
-				filter(new RichFilter1());
-
-		DataSet<Tuple3<Integer, Long, String>> unionDs = CollectionDataSets.get3TupleDataSet(env)
-				.union(empty);
-
-		List<Tuple3<Integer, Long, String>> result = unionDs.collect();
-
-		String expected = FULL_TUPLE_3_STRING;
-
-		compareResultAsTuples(result, expected);
-	}
-
-	public static class RichFilter1 extends RichFilterFunction<Tuple3<Integer,Long,String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple3<Integer, Long, String> value) throws Exception {
-			return false;
-		}
-	}
-
-}


[21/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
index bda1679..fd4ecd4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
@@ -28,8 +28,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
@@ -41,6 +41,7 @@ import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -51,13 +52,14 @@ import java.util.List;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.flink.test.util.TestUtils.tryExecute;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * This verifies that checkpointing works correctly with event time windows.
  *
- * <p>
- * This is a version of {@link AbstractEventTimeWindowCheckpointingITCase} for All-Windows.
+ * <p>This is a version of {@link AbstractEventTimeWindowCheckpointingITCase} for All-Windows.
  */
 @SuppressWarnings("serial")
 public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
@@ -68,7 +70,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	private static TestStreamEnvironment env;
 
-
 	@BeforeClass
 	public static void startTestCluster() {
 		Configuration config = new Configuration();
@@ -94,11 +95,11 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testTumblingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = 3000;
-		final int WINDOW_SIZE = 100;
-		final int NUM_KEYS = 1;
+		final int numElementsPerKey = 3000;
+		final int windowSize = 100;
+		final int numKeys = 1;
 		FailingSource.reset();
-		
+
 		try {
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -107,11 +108,11 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_KEYS,
-							NUM_ELEMENTS_PER_KEY,
-							NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys,
+							numElementsPerKey,
+							numElementsPerKey / 3))
 					.rebalance()
-					.timeWindowAll(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindowAll(Time.of(windowSize, MILLISECONDS))
 					.apply(new RichAllWindowFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, TimeWindow>() {
 
 						private boolean open = false;
@@ -141,8 +142,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 							out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -154,10 +154,10 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testSlidingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = 3000;
-		final int WINDOW_SIZE = 1000;
-		final int WINDOW_SLIDE = 100;
-		final int NUM_KEYS = 1;
+		final int numElementsPerKey = 3000;
+		final int windowSize = 1000;
+		final int windowSlide = 100;
+		final int numKeys = 1;
 		FailingSource.reset();
 
 		try {
@@ -168,9 +168,9 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys, numElementsPerKey, numElementsPerKey / 3))
 					.rebalance()
-					.timeWindowAll(Time.of(WINDOW_SIZE, MILLISECONDS), Time.of(WINDOW_SLIDE, MILLISECONDS))
+					.timeWindowAll(Time.of(windowSize, MILLISECONDS), Time.of(windowSlide, MILLISECONDS))
 					.apply(new RichAllWindowFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, TimeWindow>() {
 
 						private boolean open = false;
@@ -200,8 +200,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 							out.collect(new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum)));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SLIDE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1);
 
 			tryExecute(env, "Sliding Window Test");
 		}
@@ -213,9 +212,9 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testPreAggregatedTumblingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = 3000;
-		final int WINDOW_SIZE = 100;
-		final int NUM_KEYS = 1;
+		final int numElementsPerKey = 3000;
+		final int windowSize = 100;
+		final int numKeys = 1;
 		FailingSource.reset();
 
 		try {
@@ -226,11 +225,11 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_KEYS,
-							NUM_ELEMENTS_PER_KEY,
-							NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys,
+							numElementsPerKey,
+							numElementsPerKey / 3))
 					.rebalance()
-					.timeWindowAll(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindowAll(Time.of(windowSize, MILLISECONDS))
 					.reduce(
 							new ReduceFunction<Tuple2<Long, IntType>>() {
 
@@ -269,8 +268,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -282,9 +280,9 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testPreAggregatedFoldingTumblingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = 3000;
-		final int WINDOW_SIZE = 100;
-		final int NUM_KEYS = 1;
+		final int numElementsPerKey = 3000;
+		final int windowSize = 100;
+		final int numKeys = 1;
 		FailingSource.reset();
 
 		try {
@@ -295,11 +293,11 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_KEYS,
-							NUM_ELEMENTS_PER_KEY,
-							NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys,
+							numElementsPerKey,
+							numElementsPerKey / 3))
 					.rebalance()
-					.timeWindowAll(Time.of(WINDOW_SIZE, MILLISECONDS))
+					.timeWindowAll(Time.of(windowSize, MILLISECONDS))
 					.fold(new Tuple4<>(0L, 0L, 0L, new IntType(0)),
 							new FoldFunction<Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>>() {
 								@Override
@@ -337,8 +335,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 									}
 								}
 							})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSize)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -350,10 +347,10 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testPreAggregatedSlidingTimeWindow() {
-		final int NUM_ELEMENTS_PER_KEY = 3000;
-		final int WINDOW_SIZE = 1000;
-		final int WINDOW_SLIDE = 100;
-		final int NUM_KEYS = 1;
+		final int numElementsPerKey = 3000;
+		final int windowSize = 1000;
+		final int windowSlide = 100;
+		final int numKeys = 1;
 		FailingSource.reset();
 
 		try {
@@ -364,12 +361,12 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_KEYS,
-							NUM_ELEMENTS_PER_KEY,
-							NUM_ELEMENTS_PER_KEY / 3))
+					.addSource(new FailingSource(numKeys,
+							numElementsPerKey,
+							numElementsPerKey / 3))
 					.rebalance()
-					.timeWindowAll(Time.of(WINDOW_SIZE, MILLISECONDS),
-							Time.of(WINDOW_SLIDE, MILLISECONDS))
+					.timeWindowAll(Time.of(windowSize, MILLISECONDS),
+							Time.of(windowSlide, MILLISECONDS))
 					.reduce(
 							new ReduceFunction<Tuple2<Long, IntType>>() {
 
@@ -408,8 +405,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SLIDE)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numKeys, numElementsPerKey / windowSlide)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -419,14 +415,12 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		}
 	}
 
-
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements ListCheckpointed<Integer>, CheckpointListener
-	{
+			implements ListCheckpointed<Integer>, CheckpointListener {
 		private static volatile boolean failedBefore = false;
 
 		private final int numKeys;
@@ -467,8 +461,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 				}
 
 				if (numElementsEmitted < numElementsToEmit &&
-						(failedBefore || numElementsEmitted <= failureAfterNumElements))
-				{
+						(failedBefore || numElementsEmitted <= failureAfterNumElements)) {
 					// the function failed before, or we are in the elements before the failure
 					synchronized (ctx.getCheckpointLock()) {
 						int next = numElementsEmitted++;
@@ -579,7 +572,6 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 			assertEquals("Window start: " + value.f1 + " end: " + value.f2, expectedSum, value.f3.value);
 
-
 			Integer curr = windowCounts.get(value.f0);
 			if (curr != null) {
 				windowCounts.put(value.f0, curr + 1);
@@ -625,12 +617,17 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
+	/**
+	 * Custom boxed integer type.
+	 */
 	public static class IntType {
 
 		public int value;
 
 		public IntType() {}
 
-		public IntType(int value) { this.value = value; }
+		public IntType(int value) {
+			this.value = value;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/FileBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/FileBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/FileBackendEventTimeWindowCheckpointingITCase.java
index 65fda09..030c1a3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/FileBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/FileBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for file backend.
+ */
 public class FileBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public FileBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java
index 352f9f7..dfb66cc 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for incremental RocksDB backend.
+ */
 public class IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public IncrementalRocksDbBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java
index 147d385..76a18c3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java
@@ -60,11 +60,11 @@ import static org.junit.Assert.fail;
 
 /**
  * A simple test that runs a streaming topology with checkpointing enabled.
- * 
- * The test triggers a failure after a while and verifies that, after
+ *
+ * <p>The test triggers a failure after a while and verifies that, after
  * completion, the state reflects the "exactly once" semantics.
- * 
- * It is designed to check partitioned states.
+ *
+ * <p>It is designed to check partitioned states.
  */
 @SuppressWarnings("serial")
 public class KeyedStateCheckpointingITCase extends TestLogger {
@@ -197,7 +197,7 @@ public class KeyedStateCheckpointingITCase extends TestLogger {
 	 * A source that generates a sequence of integers and throttles down until a checkpoint
 	 * has happened.
 	 */
-	private static class IntGeneratingSourceFunction extends RichParallelSourceFunction<Integer> 
+	private static class IntGeneratingSourceFunction extends RichParallelSourceFunction<Integer>
 		implements ListCheckpointed<Integer>, CheckpointListener {
 
 		private final int numElements;
@@ -363,7 +363,7 @@ public class KeyedStateCheckpointingITCase extends TestLogger {
 		}
 	}
 
-	public static class IdentityKeySelector<T> implements KeySelector<T, T> {
+	private static class IdentityKeySelector<T> implements KeySelector<T, T> {
 
 		@Override
 		public T getKey(T value) throws Exception {
@@ -375,6 +375,9 @@ public class KeyedStateCheckpointingITCase extends TestLogger {
 	//  data types
 	// ------------------------------------------------------------------------
 
+	/**
+	 * Custom boxed long type that does not implement Serializable.
+	 */
 	public static class NonSerializableLong {
 
 		public long value;
@@ -389,7 +392,7 @@ public class KeyedStateCheckpointingITCase extends TestLogger {
 
 		@Override
 		public boolean equals(Object obj) {
-			return this == obj || 
+			return this == obj ||
 					obj != null && obj.getClass() == getClass() && ((NonSerializableLong) obj).value == this.value;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/MemBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/MemBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/MemBackendEventTimeWindowCheckpointingITCase.java
index 899b8d6..54a29ed 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/MemBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/MemBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for memory backend.
+ */
 public class MemBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public MemBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
index 264b22e..cad6693 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
@@ -55,7 +55,6 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 
-import io.netty.util.internal.ConcurrentSet;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -85,6 +84,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test savepoint rescaling.
+ */
 @RunWith(Parameterized.class)
 public class RescalingITCase extends TestLogger {
 
@@ -250,7 +252,6 @@ public class RescalingITCase extends TestLogger {
 
 			assertEquals(expectedResult2, actualResult2);
 
-
 		} finally {
 			// clear the CollectionSink set for the restarted job
 			CollectionSink.clearElementsSet();
@@ -502,7 +503,6 @@ public class RescalingITCase extends TestLogger {
 		testSavepointRescalingPartitionedOperatorState(true, OperatorCheckpointMethod.LIST_CHECKPOINTED);
 	}
 
-
 	/**
 	 * Tests rescaling of partitioned operator state. More specific, we test the mechanism with {@link ListCheckpointed}
 	 * as it subsumes {@link org.apache.flink.streaming.api.checkpoint.CheckpointedFunction}.
@@ -522,11 +522,11 @@ public class RescalingITCase extends TestLogger {
 
 		if (checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION ||
 				checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION_BROADCAST) {
-			PartitionedStateSource.CHECK_CORRECT_SNAPSHOT = new int[counterSize];
-			PartitionedStateSource.CHECK_CORRECT_RESTORE = new int[counterSize];
+			PartitionedStateSource.checkCorrectSnapshot = new int[counterSize];
+			PartitionedStateSource.checkCorrectRestore = new int[counterSize];
 		} else {
-			PartitionedStateSourceListCheckpointed.CHECK_CORRECT_SNAPSHOT = new int[counterSize];
-			PartitionedStateSourceListCheckpointed.CHECK_CORRECT_RESTORE = new int[counterSize];
+			PartitionedStateSourceListCheckpointed.checkCorrectSnapshot = new int[counterSize];
+			PartitionedStateSourceListCheckpointed.checkCorrectRestore = new int[counterSize];
 		}
 
 		try {
@@ -584,29 +584,29 @@ public class RescalingITCase extends TestLogger {
 			int sumAct = 0;
 
 			if (checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION) {
-				for (int c : PartitionedStateSource.CHECK_CORRECT_SNAPSHOT) {
+				for (int c : PartitionedStateSource.checkCorrectSnapshot) {
 					sumExp += c;
 				}
 
-				for (int c : PartitionedStateSource.CHECK_CORRECT_RESTORE) {
+				for (int c : PartitionedStateSource.checkCorrectRestore) {
 					sumAct += c;
 				}
 			} else if (checkpointMethod == OperatorCheckpointMethod.CHECKPOINTED_FUNCTION_BROADCAST) {
-				for (int c : PartitionedStateSource.CHECK_CORRECT_SNAPSHOT) {
+				for (int c : PartitionedStateSource.checkCorrectSnapshot) {
 					sumExp += c;
 				}
 
-				for (int c : PartitionedStateSource.CHECK_CORRECT_RESTORE) {
+				for (int c : PartitionedStateSource.checkCorrectRestore) {
 					sumAct += c;
 				}
 
 				sumExp *= parallelism2;
 			} else {
-				for (int c : PartitionedStateSourceListCheckpointed.CHECK_CORRECT_SNAPSHOT) {
+				for (int c : PartitionedStateSourceListCheckpointed.checkCorrectSnapshot) {
 					sumExp += c;
 				}
 
-				for (int c : PartitionedStateSourceListCheckpointed.CHECK_CORRECT_RESTORE) {
+				for (int c : PartitionedStateSourceListCheckpointed.checkCorrectRestore) {
 					sumAct += c;
 				}
 			}
@@ -777,8 +777,8 @@ public class RescalingITCase extends TestLogger {
 				if (counter < numberElements) {
 					synchronized (lock) {
 						for (int value = subtaskIndex;
-						     value < numberKeys;
-						     value += getRuntimeContext().getNumberOfParallelSubtasks()) {
+							value < numberKeys;
+							value += getRuntimeContext().getNumberOfParallelSubtasks()) {
 
 							ctx.collect(value);
 						}
@@ -943,13 +943,13 @@ public class RescalingITCase extends TestLogger {
 		private static final long serialVersionUID = -4357864582992546L;
 		private static final int NUM_PARTITIONS = 7;
 
-		private static int[] CHECK_CORRECT_SNAPSHOT;
-		private static int[] CHECK_CORRECT_RESTORE;
+		private static int[] checkCorrectSnapshot;
+		private static int[] checkCorrectRestore;
 
 		@Override
 		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
 
-			CHECK_CORRECT_SNAPSHOT[getRuntimeContext().getIndexOfThisSubtask()] = counter;
+			checkCorrectSnapshot[getRuntimeContext().getIndexOfThisSubtask()] = counter;
 
 			int div = counter / NUM_PARTITIONS;
 			int mod = counter % NUM_PARTITIONS;
@@ -971,7 +971,7 @@ public class RescalingITCase extends TestLogger {
 			for (Integer v : state) {
 				counter += v;
 			}
-			CHECK_CORRECT_RESTORE[getRuntimeContext().getIndexOfThisSubtask()] = counter;
+			checkCorrectRestore[getRuntimeContext().getIndexOfThisSubtask()] = counter;
 		}
 	}
 
@@ -983,8 +983,8 @@ public class RescalingITCase extends TestLogger {
 		private transient ListState<Integer> counterPartitions;
 		private boolean broadcast;
 
-		private static int[] CHECK_CORRECT_SNAPSHOT;
-		private static int[] CHECK_CORRECT_RESTORE;
+		private static int[] checkCorrectSnapshot;
+		private static int[] checkCorrectRestore;
 
 		public PartitionedStateSource(boolean broadcast) {
 			this.broadcast = broadcast;
@@ -995,7 +995,7 @@ public class RescalingITCase extends TestLogger {
 
 			counterPartitions.clear();
 
-			CHECK_CORRECT_SNAPSHOT[getRuntimeContext().getIndexOfThisSubtask()] = counter;
+			checkCorrectSnapshot[getRuntimeContext().getIndexOfThisSubtask()] = counter;
 
 			int div = counter / NUM_PARTITIONS;
 			int mod = counter % NUM_PARTITIONS;
@@ -1027,7 +1027,7 @@ public class RescalingITCase extends TestLogger {
 				for (int v : counterPartitions.get()) {
 					counter += v;
 				}
-				CHECK_CORRECT_RESTORE[getRuntimeContext().getIndexOfThisSubtask()] = counter;
+				checkCorrectRestore[getRuntimeContext().getIndexOfThisSubtask()] = counter;
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RocksDbBackendEventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RocksDbBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RocksDbBackendEventTimeWindowCheckpointingITCase.java
index da2bbc7..3873aff 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RocksDbBackendEventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RocksDbBackendEventTimeWindowCheckpointingITCase.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.test.checkpointing;
 
+/**
+ * Integration tests for fully synchronous RocksDB backend.
+ */
 public class RocksDbBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase {
 
 	public RocksDbBackendEventTimeWindowCheckpointingITCase() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index 09dfa99..a3d45dd 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -18,11 +18,6 @@
 
 package org.apache.flink.test.checkpointing;
 
-import akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.common.functions.MapFunction;
@@ -78,17 +73,18 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -103,6 +99,12 @@ import java.util.Random;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
+import scala.Option;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
 import static org.apache.flink.runtime.messages.JobManagerMessages.getDisposeSavepointSuccess;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -479,7 +481,7 @@ public class SavepointITCase extends TestLogger {
 	/**
 	 * FLINK-5985
 	 *
-	 * This test ensures we can restore from a savepoint under modifications to the job graph that only concern
+	 * <p>This test ensures we can restore from a savepoint under modifications to the job graph that only concern
 	 * stateless operators.
 	 */
 	@Test
@@ -736,17 +738,17 @@ public class SavepointITCase extends TestLogger {
 	}
 
 	private static final int ITER_TEST_PARALLELISM = 1;
-	private static OneShotLatch[] ITER_TEST_SNAPSHOT_WAIT = new OneShotLatch[ITER_TEST_PARALLELISM];
-	private static OneShotLatch[] ITER_TEST_RESTORE_WAIT = new OneShotLatch[ITER_TEST_PARALLELISM];
-	private static int[] ITER_TEST_CHECKPOINT_VERIFY = new int[ITER_TEST_PARALLELISM];
+	private static OneShotLatch[] iterTestSnapshotWait = new OneShotLatch[ITER_TEST_PARALLELISM];
+	private static OneShotLatch[] iterTestRestoreWait = new OneShotLatch[ITER_TEST_PARALLELISM];
+	private static int[] iterTestCheckpointVerify = new int[ITER_TEST_PARALLELISM];
 
 	@Test
 	public void testSavepointForJobWithIteration() throws Exception {
 
 		for (int i = 0; i < ITER_TEST_PARALLELISM; ++i) {
-			ITER_TEST_SNAPSHOT_WAIT[i] = new OneShotLatch();
-			ITER_TEST_RESTORE_WAIT[i] = new OneShotLatch();
-			ITER_TEST_CHECKPOINT_VERIFY[i] = 0;
+			iterTestSnapshotWait[i] = new OneShotLatch();
+			iterTestRestoreWait[i] = new OneShotLatch();
+			iterTestCheckpointVerify[i] = 0;
 		}
 
 		TemporaryFolder folder = new TemporaryFolder();
@@ -821,7 +823,7 @@ public class SavepointITCase extends TestLogger {
 			cluster.start();
 
 			cluster.submitJobDetached(jobGraph);
-			for (OneShotLatch latch : ITER_TEST_SNAPSHOT_WAIT) {
+			for (OneShotLatch latch : iterTestSnapshotWait) {
 				latch.await();
 			}
 			savepointPath = cluster.triggerSavepoint(jobGraph.getJobID());
@@ -831,7 +833,7 @@ public class SavepointITCase extends TestLogger {
 			jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath));
 
 			cluster.submitJobDetached(jobGraph);
-			for (OneShotLatch latch : ITER_TEST_RESTORE_WAIT) {
+			for (OneShotLatch latch : iterTestRestoreWait) {
 				latch.await();
 			}
 			source.cancel();
@@ -883,7 +885,7 @@ public class SavepointITCase extends TestLogger {
 
 		@Override
 		public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
-			ITER_TEST_CHECKPOINT_VERIFY[getRuntimeContext().getIndexOfThisSubtask()] = emittedCount;
+			iterTestCheckpointVerify[getRuntimeContext().getIndexOfThisSubtask()] = emittedCount;
 			return Collections.singletonList(emittedCount);
 		}
 
@@ -892,20 +894,20 @@ public class SavepointITCase extends TestLogger {
 			if (!state.isEmpty()) {
 				this.emittedCount = state.get(0);
 			}
-			Assert.assertEquals(ITER_TEST_CHECKPOINT_VERIFY[getRuntimeContext().getIndexOfThisSubtask()], emittedCount);
-			ITER_TEST_RESTORE_WAIT[getRuntimeContext().getIndexOfThisSubtask()].trigger();
+			Assert.assertEquals(iterTestCheckpointVerify[getRuntimeContext().getIndexOfThisSubtask()], emittedCount);
+			iterTestRestoreWait[getRuntimeContext().getIndexOfThisSubtask()].trigger();
 		}
 	}
 
-	public static class DuplicateFilter extends RichFlatMapFunction<Integer, Integer> {
+	private static class DuplicateFilter extends RichFlatMapFunction<Integer, Integer> {
 
-		static final ValueStateDescriptor<Boolean> descriptor = new ValueStateDescriptor<>("seen", Boolean.class, false);
+		static final ValueStateDescriptor<Boolean> DESCRIPTOR = new ValueStateDescriptor<>("seen", Boolean.class, false);
 		private static final long serialVersionUID = 1L;
 		private ValueState<Boolean> operatorState;
 
 		@Override
 		public void open(Configuration configuration) {
-			operatorState = this.getRuntimeContext().getState(descriptor);
+			operatorState = this.getRuntimeContext().getState(DESCRIPTOR);
 		}
 
 		@Override
@@ -916,7 +918,7 @@ public class SavepointITCase extends TestLogger {
 			}
 
 			if (30 == value) {
-				ITER_TEST_SNAPSHOT_WAIT[getRuntimeContext().getIndexOfThisSubtask()].trigger();
+				iterTestSnapshotWait[getRuntimeContext().getIndexOfThisSubtask()].trigger();
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
index 32d9e23..0fcfb8f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java
@@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.util.Collector;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,11 +47,11 @@ import static org.junit.Assert.assertTrue;
 /**
  * A simple test that runs a streaming topology with checkpointing enabled.
  *
- * The test triggers a failure after a while and verifies that, after completion, the
+ * <p>The test triggers a failure after a while and verifies that, after completion, the
  * state defined with either the {@link ValueState} or the {@link ListCheckpointed}
  * interface reflects the "exactly once" semantics.
- * 
- * The test throttles the input until at least two checkpoints are completed, to make sure that
+ *
+ * <p>The test throttles the input until at least two checkpoints are completed, to make sure that
  * the recovery does not fall back to "square one" (which would naturally lead to correct
  * results without testing the checkpointing).
  */
@@ -59,11 +60,10 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 
 	private static final Logger LOG = LoggerFactory.getLogger(StateCheckpointedITCase.class);
 
-	final long NUM_STRINGS = 10_000_000L;
+	static final long NUM_STRINGS = 10_000_000L;
 
 	/**
-	 * Runs the following program:
-	 *
+	 * Runs the following program.
 	 * <pre>
 	 *     [ (source)->(filter)] -> [ (map) -> (map) ] -> [ (groupBy/reduce)->(sink) ]
 	 * </pre>
@@ -84,7 +84,7 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		stream
 				// first vertex, chained to the source
 				// this filter throttles the flow until at least one checkpoint
-				// is complete, to make sure this program does not run without 
+				// is complete, to make sure this program does not run without
 				.filter(new StringRichFilterFunction())
 
 						// -------------- seconds vertex - one-to-one connected ----------------
@@ -100,13 +100,13 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 
 	@Override
 	public void postSubmit() {
-		
+
 		//assertTrue("Test inconclusive: failure occurred before first checkpoint",
 		//		OnceFailingAggregator.wasCheckpointedBeforeFailure);
-		if(!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
+		if (!OnceFailingAggregator.wasCheckpointedBeforeFailure) {
 			LOG.warn("Test inconclusive: failure occurred before first checkpoint");
 		}
-		
+
 		long filterSum = 0;
 		for (long l : StringRichFilterFunction.counts) {
 			filterSum += l;
@@ -137,10 +137,9 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 	// --------------------------------------------------------------------------------------------
 	//  Custom Functions
 	// --------------------------------------------------------------------------------------------
-	
-	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String> 
-			implements ListCheckpointed<Integer>
-	{
+
+	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
+			implements ListCheckpointed<Integer> {
 		private final long numElements;
 
 		private int index;
@@ -157,9 +156,9 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 
 			final Random rnd = new Random();
 			final StringBuilder stringBuilder = new StringBuilder();
-			
+
 			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
-			
+
 			if (index == 0) {
 				index = getRuntimeContext().getIndexOfThisSubtask();
 			}
@@ -178,7 +177,7 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 				}
 			}
 		}
-		
+
 		@Override
 		public void cancel() {
 			isRunning = false;
@@ -209,11 +208,11 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 	}
 
-	private static class StringRichFilterFunction extends RichFilterFunction<String> 
+	private static class StringRichFilterFunction extends RichFilterFunction<String>
 			implements ListCheckpointed<Long> {
 
-		static final long[] counts = new long[PARALLELISM];
-		
+		static long[] counts = new long[PARALLELISM];
+
 		private long count;
 
 		@Override
@@ -241,10 +240,10 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 	}
 
-	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount> 
+	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount>
 			implements ListCheckpointed<Long> {
-		
-		static final long[] counts = new long[PARALLELISM];
+
+		static long[] counts = new long[PARALLELISM];
 
 		private long count;
 
@@ -272,12 +271,12 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 			this.count = state.get(0);
 		}
 	}
-	
-	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> 
+
+	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount>
 		implements ListCheckpointed<Long> {
 
-		static final long[] counts = new long[PARALLELISM];
-		
+		static long[] counts = new long[PARALLELISM];
+
 		private long count;
 
 		@Override
@@ -304,25 +303,25 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 			this.count = state.get(0);
 		}
 	}
-	
-	private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount> 
+
+	private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount>
 		implements ListCheckpointed<HashMap<String, PrefixCount>>, CheckpointListener {
 
 		static boolean wasCheckpointedBeforeFailure = false;
-		
+
 		private static volatile boolean hasFailed = false;
 
 		private final HashMap<String, PrefixCount> aggregationMap = new HashMap<String, PrefixCount>();
-		
+
 		private long failurePos;
 		private long count;
-		
+
 		private boolean wasCheckpointed;
 
 		OnceFailingAggregator(long failurePos) {
 			this.failurePos = failurePos;
 		}
-		
+
 		@Override
 		public void open(Configuration parameters) {
 			count = 0;
@@ -336,7 +335,7 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 				hasFailed = true;
 				throw new Exception("Test Failure");
 			}
-			
+
 			PrefixCount curr = aggregationMap.get(value.prefix);
 			if (curr == null) {
 				aggregationMap.put(value.prefix, value);
@@ -367,12 +366,12 @@ public class StateCheckpointedITCase extends StreamFaultToleranceTestBase {
 		}
 	}
 
-	private static class ValidatingSink extends RichSinkFunction<PrefixCount> 
+	private static class ValidatingSink extends RichSinkFunction<PrefixCount>
 			implements ListCheckpointed<HashMap<Character, Long>> {
 
 		@SuppressWarnings("unchecked")
 		private static Map<Character, Long>[] maps = (Map<Character, Long>[]) new Map<?, ?>[PARALLELISM];
-		
+
 		private HashMap<Character, Long> counts = new HashMap<Character, Long>();
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
index d76d674..16d8b54 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
@@ -38,7 +38,6 @@ import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
 
 import org.junit.Test;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,12 +61,10 @@ import static org.junit.Assert.fail;
  * checkpoints, that it is called at most once for any checkpoint id and that it is not
  * called for a deliberately failed checkpoint.
  *
- * <p>
- * The topology tested here includes a number of {@link OneInputStreamOperator}s and a
+ * <p>The topology tested here includes a number of {@link OneInputStreamOperator}s and a
  * {@link TwoInputStreamOperator}.
  *
- * <p>
- * Note that as a result of doing the checks on the task level there is no way to verify
+ * <p>Note that as a result of doing the checks on the task level there is no way to verify
  * that the {@link CheckpointListener#notifyCheckpointComplete(long)} is called for every
  * successfully completed checkpoint.
  */
@@ -79,8 +76,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	private static final int PARALLELISM = 4;
 
 	/**
-	 * Runs the following program:
-	 *
+	 * Runs the following program.
 	 * <pre>
 	 *     [ (source)->(filter) ] -> [ (co-map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
 	 * </pre>
@@ -95,52 +91,52 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 			env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L));
 
 			final int numElements = 10000;
-			final int numTaskTotal = PARALLELISM * 5; 
+			final int numTaskTotal = PARALLELISM * 5;
 
 			DataStream<Long> stream = env.addSource(new GeneratingSourceFunction(numElements, numTaskTotal));
 
 			stream
 					// -------------- first vertex, chained to the src ----------------
 					.filter(new LongRichFilterFunction())
-	
+
 					// -------------- second vertex, applying the co-map ----------------
 					.connect(stream).flatMap(new LeftIdentityCoRichFlatMapFunction())
-	
+
 					// -------------- third vertex - the stateful one that also fails ----------------
 					.map(new IdentityMapFunction())
 					.startNewChain()
-	
+
 					// -------------- fourth vertex - reducer and the sink ----------------
 					.keyBy(0)
 					.reduce(new OnceFailingReducer(numElements))
-				
+
 					.addSink(new DiscardingSink<Tuple1<Long>>());
-			
+
 			env.execute();
 
 			final long failureCheckpointID = OnceFailingReducer.failureCheckpointID;
 			assertNotEquals(0L, failureCheckpointID);
-			
+
 			List<List<Long>[]> allLists = Arrays.asList(
-				GeneratingSourceFunction.completedCheckpoints,
-				LongRichFilterFunction.completedCheckpoints,
-				LeftIdentityCoRichFlatMapFunction.completedCheckpoints,
-				IdentityMapFunction.completedCheckpoints,
-				OnceFailingReducer.completedCheckpoints
+				GeneratingSourceFunction.COMPLETED_CHECKPOINTS,
+				LongRichFilterFunction.COMPLETED_CHECKPOINTS,
+				LeftIdentityCoRichFlatMapFunction.COMPLETED_CHECKPOINTS,
+				IdentityMapFunction.COMPLETED_CHECKPOINTS,
+				OnceFailingReducer.COMPLETED_CHECKPOINTS
 			);
 
 			for (List<Long>[] parallelNotifications : allLists) {
 				for (List<Long> notifications : parallelNotifications) {
-					
-					assertTrue("No checkpoint notification was received.", 
+
+					assertTrue("No checkpoint notification was received.",
 						notifications.size() > 0);
-					
+
 					assertFalse("Failure checkpoint was marked as completed.",
 						notifications.contains(failureCheckpointID));
-					
+
 					assertFalse("No checkpoint received after failure.",
 						notifications.get(notifications.size() - 1) == failureCheckpointID);
-					
+
 					assertTrue("Checkpoint notification was received multiple times",
 						notifications.size() == new HashSet<Long>(notifications).size());
 				}
@@ -160,7 +156,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		}
 		return lists;
 	}
-	
+
 	// --------------------------------------------------------------------------------------------
 	//  Custom Functions
 	// --------------------------------------------------------------------------------------------
@@ -171,21 +167,21 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	 */
 	private static class GeneratingSourceFunction extends RichSourceFunction<Long>
 			implements ParallelSourceFunction<Long>, CheckpointListener, ListCheckpointed<Integer> {
-		
-		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
-		
+
+		static final List<Long>[] COMPLETED_CHECKPOINTS = createCheckpointLists(PARALLELISM);
+
 		static AtomicLong numPostFailureNotifications = new AtomicLong();
 
 		// operator behaviour
 		private final long numElements;
-		
+
 		private final int notificationsToWaitFor;
 
 		private int index;
 		private int step;
 
 		private volatile boolean notificationAlready;
-		
+
 		private volatile boolean isRunning = true;
 
 		GeneratingSourceFunction(long numElements, int notificationsToWaitFor) {
@@ -198,8 +194,9 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 			step = getRuntimeContext().getNumberOfParallelSubtasks();
 
 			// if index has been restored, it is not 0 any more
-			if (index == 0)
+			if (index == 0) {
 				index = getRuntimeContext().getIndexOfThisSubtask();
+			}
 		}
 
 		@Override
@@ -214,7 +211,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 					ctx.collect(result);
 				}
 			}
-			
+
 			// if the program goes fast and no notifications come through, we
 			// wait until all tasks had a chance to see a notification
 			while (isRunning && numPostFailureNotifications.get() < notificationsToWaitFor) {
@@ -244,7 +241,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		public void notifyCheckpointComplete(long checkpointId) {
 			// record the ID of the completed checkpoint
 			int partition = getRuntimeContext().getIndexOfThisSubtask();
-			completedCheckpoints[partition].add(checkpointId);
+			COMPLETED_CHECKPOINTS[partition].add(checkpointId);
 
 			// if this is the first time we get a notification since the failure,
 			// tell the source function
@@ -262,7 +259,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	private static class IdentityMapFunction extends RichMapFunction<Long, Tuple1<Long>>
 			implements CheckpointListener {
 
-		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
+		static final List<Long>[] COMPLETED_CHECKPOINTS = createCheckpointLists(PARALLELISM);
 
 		private volatile boolean notificationAlready;
 
@@ -275,7 +272,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		public void notifyCheckpointComplete(long checkpointId) {
 			// record the ID of the completed checkpoint
 			int partition = getRuntimeContext().getIndexOfThisSubtask();
-			completedCheckpoints[partition].add(checkpointId);
+			COMPLETED_CHECKPOINTS[partition].add(checkpointId);
 
 			// if this is the first time we get a notification since the failure,
 			// tell the source function
@@ -293,10 +290,10 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	 */
 	private static class LongRichFilterFunction extends RichFilterFunction<Long> implements CheckpointListener {
 
-		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
-		
+		static final List<Long>[] COMPLETED_CHECKPOINTS = createCheckpointLists(PARALLELISM);
+
 		private volatile boolean notificationAlready;
-		
+
 		@Override
 		public boolean filter(Long value) {
 			return value < 100;
@@ -306,8 +303,8 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		public void notifyCheckpointComplete(long checkpointId) {
 			// record the ID of the completed checkpoint
 			int partition = getRuntimeContext().getIndexOfThisSubtask();
-			completedCheckpoints[partition].add(checkpointId);
-			
+			COMPLETED_CHECKPOINTS[partition].add(checkpointId);
+
 			// if this is the first time we get a notification since the failure,
 			// tell the source function
 			if (OnceFailingReducer.hasFailed && !notificationAlready) {
@@ -325,10 +322,10 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction<Long, Long, Long>
 			implements CheckpointListener {
 
-		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
+		static final List<Long>[] COMPLETED_CHECKPOINTS = createCheckpointLists(PARALLELISM);
 
 		private volatile boolean notificationAlready;
-		
+
 		@Override
 		public void flatMap1(Long value, Collector<Long> out) {
 			out.collect(value);
@@ -343,7 +340,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		public void notifyCheckpointComplete(long checkpointId) {
 			// record the ID of the completed checkpoint
 			int partition = getRuntimeContext().getIndexOfThisSubtask();
-			completedCheckpoints[partition].add(checkpointId);
+			COMPLETED_CHECKPOINTS[partition].add(checkpointId);
 
 			// if this is the first time we get a notification since the failure,
 			// tell the source function
@@ -357,16 +354,15 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 	/**
 	 * Reducer that causes one failure between seeing 40% to 70% of the records.
 	 */
-	private static class OnceFailingReducer extends RichReduceFunction<Tuple1<Long>> 
-		implements ListCheckpointed<Long>, CheckpointListener
-	{
+	private static class OnceFailingReducer extends RichReduceFunction<Tuple1<Long>>
+		implements ListCheckpointed<Long>, CheckpointListener {
 		static volatile boolean hasFailed = false;
 		static volatile long failureCheckpointID;
 
-		static final List<Long>[] completedCheckpoints = createCheckpointLists(PARALLELISM);
-		
+		static final List<Long>[] COMPLETED_CHECKPOINTS = createCheckpointLists(PARALLELISM);
+
 		private final long failurePos;
-		
+
 		private volatile long count;
 
 		private volatile boolean notificationAlready;
@@ -381,7 +377,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 			if (count >= failurePos && getRuntimeContext().getIndexOfThisSubtask() == 0) {
 				LOG.info(">>>>>>>>>>>>>>>>> Reached failing position <<<<<<<<<<<<<<<<<<<<<");
 			}
-			
+
 			value1.f0 += value2.f0;
 			return value1;
 		}
@@ -409,7 +405,7 @@ public class StreamCheckpointNotifierITCase extends StreamingMultipleProgramsTes
 		public void notifyCheckpointComplete(long checkpointId) {
 			// record the ID of the completed checkpoint
 			int partition = getRuntimeContext().getIndexOfThisSubtask();
-			completedCheckpoints[partition].add(checkpointId);
+			COMPLETED_CHECKPOINTS[partition].add(checkpointId);
 
 			// if this is the first time we get a notification since the failure,
 			// tell the source function

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
index aae04c9..616e794 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
@@ -41,18 +41,17 @@ import static org.junit.Assert.assertEquals;
 
 /**
  * A simple test that runs a streaming topology with checkpointing enabled.
- * 
- * The test triggers a failure after a while and verifies that, after completion, the
+ *
+ * <p>The test triggers a failure after a while and verifies that, after completion, the
  * state defined with the {@link ListCheckpointed} interface reflects the "exactly once" semantics.
  */
 @SuppressWarnings("serial")
 public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 
-	final long NUM_STRINGS = 10_000_000L;
+	static final long NUM_STRINGS = 10_000_000L;
 
 	/**
-	 * Runs the following program:
-	 *
+	 * Runs the following program.
 	 * <pre>
 	 *     [ (source)->(filter) ]-s->[ (map) ] -> [ (map) ] -> [ (groupBy/count)->(sink) ]
 	 * </pre>
@@ -101,7 +100,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		}
 
 		long reduceInputCount = 0;
-		for(long l: OnceFailingPrefixCounter.counts){
+		for (long l: OnceFailingPrefixCounter.counts){
 			reduceInputCount += l;
 		}
 
@@ -118,12 +117,12 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 	// --------------------------------------------------------------------------------------------
 	//  Custom Functions
 	// --------------------------------------------------------------------------------------------
-	
+
 	private static class StringGeneratingSourceFunction extends RichSourceFunction<String>
 			implements ParallelSourceFunction<String>, ListCheckpointed<Integer> {
 
 		private final long numElements;
-		
+
 		private final Random rnd = new Random();
 		private final StringBuilder stringBuilder = new StringBuilder();
 
@@ -132,14 +131,13 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 
 		private volatile boolean isRunning = true;
 
-		static final long[] counts = new long[PARALLELISM];
+		static long[] counts = new long[PARALLELISM];
 
 		@Override
 		public void close() throws IOException {
 			counts[getRuntimeContext().getIndexOfThisSubtask()] = index;
 		}
 
-
 		StringGeneratingSourceFunction(long numElements) {
 			this.numElements = numElements;
 		}
@@ -200,11 +198,11 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 			this.index = state.get(0);
 		}
 	}
-	
+
 	private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> implements ListCheckpointed<Long> {
 
 		private long count;
-		static final long[] counts = new long[PARALLELISM];
+		static long[] counts = new long[PARALLELISM];
 
 		@Override
 		public PrefixCount map(PrefixCount value) throws Exception {
@@ -234,16 +232,16 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 	/**
 	 * This function uses simultaneously the key/value state and is checkpointed.
 	 */
-	private static class OnceFailingPrefixCounter extends RichMapFunction<PrefixCount, PrefixCount> 
+	private static class OnceFailingPrefixCounter extends RichMapFunction<PrefixCount, PrefixCount>
 			implements ListCheckpointed<Long> {
-		
+
 		private static Map<String, Long> prefixCounts = new ConcurrentHashMap<String, Long>();
-		static final long[] counts = new long[PARALLELISM];
+		static long[] counts = new long[PARALLELISM];
 
 		private static volatile boolean hasFailed = false;
 
 		private final long numElements;
-		
+
 		private long failurePos;
 		private long count;
 
@@ -253,7 +251,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 		OnceFailingPrefixCounter(long numElements) {
 			this.numElements = numElements;
 		}
-		
+
 		@Override
 		public void open(Configuration parameters) throws IOException {
 			long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
@@ -261,10 +259,10 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 
 			failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
 			count = 0;
-			
+
 			pCount = getRuntimeContext().getState(new ValueStateDescriptor<>("pCount", Long.class, 0L));
 		}
-		
+
 		@Override
 		public void close() throws IOException {
 			counts[getRuntimeContext().getIndexOfThisSubtask()] = inputCount;
@@ -278,7 +276,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 				throw new Exception("Test Failure");
 			}
 			inputCount++;
-		
+
 			long currentPrefixCount = pCount.value() + value.count;
 			pCount.update(currentPrefixCount);
 			prefixCounts.put(value.prefix, currentPrefixCount);
@@ -301,8 +299,8 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 	}
 
 	private static class StringRichFilterFunction extends RichFilterFunction<String> implements ListCheckpointed<Long> {
-		
-		static final long[] counts = new long[PARALLELISM];
+
+		static long[] counts = new long[PARALLELISM];
 
 		private long count;
 
@@ -333,8 +331,8 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
 
 	private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount>
 			implements ListCheckpointed<Long> {
-		
-		static final long[] counts = new long[PARALLELISM];
+
+		static long[] counts = new long[PARALLELISM];
 
 		private long count;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
index 5f56def..5d902ff 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
@@ -38,7 +38,7 @@ import java.io.Serializable;
 import static org.junit.Assert.fail;
 
 /**
- * Test base for fault tolerant streaming programs
+ * Test base for fault tolerant streaming programs.
  */
 public abstract class StreamFaultToleranceTestBase extends TestLogger {
 
@@ -55,7 +55,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
 			config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L);
-			
+
 			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
@@ -82,12 +82,12 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	 * Implementations are expected to assemble the test topology in this function
 	 * using the provided {@link StreamExecutionEnvironment}.
 	 */
-	abstract public void testProgram(StreamExecutionEnvironment env);
+	public abstract void testProgram(StreamExecutionEnvironment env);
 
 	/**
 	 * Implementations are expected to provide test here to verify the correct behavior.
 	 */
-	abstract public void postSubmit() throws Exception ;
+	public abstract void postSubmit() throws Exception;
 
 	/**
 	 * Runs the following program the test program defined in {@link #testProgram(StreamExecutionEnvironment)}
@@ -118,6 +118,9 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	//  Frequently used utilities
 	// --------------------------------------------------------------------------------------------
 
+	/**
+	 * POJO storing prefix, value, and count.
+	 */
 	@SuppressWarnings("serial")
 	public static class PrefixCount implements Serializable {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/TimestampedFileInputSplitTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/TimestampedFileInputSplitTest.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/TimestampedFileInputSplitTest.java
index 0a89ab9..1cf5829 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/TimestampedFileInputSplitTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/TimestampedFileInputSplitTest.java
@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.checkpointing;
 
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -27,6 +29,9 @@ import java.util.List;
 import java.util.PriorityQueue;
 import java.util.Queue;
 
+/**
+ * Test the {@link TimestampedFileInputSplit} for Continuous File Processing.
+ */
 public class TimestampedFileInputSplitTest {
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
index a219b68..f19d690 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.checkpointing;
 
-import com.google.common.collect.EvictingQueue;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
@@ -33,6 +32,8 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamGroupedFold;
 import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
+
+import com.google.common.collect.EvictingQueue;
 import org.junit.Assert;
 
 import java.util.Collections;
@@ -45,15 +46,14 @@ import java.util.Random;
  * of {@link AbstractUdfStreamOperator} is correctly restored in case of recovery from
  * a failure.
  *
- * <p>
- * The topology currently tests the proper behaviour of the {@link StreamGroupedReduce}
+ * <p>The topology currently tests the proper behaviour of the {@link StreamGroupedReduce}
  * and the {@link StreamGroupedFold} operators.
  */
 @SuppressWarnings("serial")
 public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTestBase {
 
-	final private static long NUM_INPUT = 500_000L;
-	final private static int NUM_OUTPUT = 1_000;
+	private static final long NUM_INPUT = 500_000L;
+	private static final int NUM_OUTPUT = 1_000;
 
 	/**
 	 * Assembles a stream of a grouping field and some long data. Applies reduce functions
@@ -66,7 +66,6 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
 		KeyedStream<Tuple2<Integer, Long>, Tuple> stream = env.addSource(new StatefulMultipleSequence())
 				.keyBy(0);
 
-
 		stream
 				// testing built-in aggregate
 				.min(1)
@@ -184,7 +183,7 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
 	 * Mapper that causes one failure between seeing 40% to 70% of the records.
 	 */
 	private static class OnceFailingIdentityMapFunction
-			extends RichMapFunction<Tuple2<Integer, Long>, Tuple2<Integer, Long>> 
+			extends RichMapFunction<Tuple2<Integer, Long>, Tuple2<Integer, Long>>
 			implements ListCheckpointed<Long> {
 
 		private static volatile boolean hasFailed = false;
@@ -223,7 +222,7 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
 
 		@Override
 		public void restoreState(List<Long> state) throws Exception {
-			if(!state.isEmpty()) {
+			if (!state.isEmpty()) {
 				count = state.get(0);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
index 7004f75..7ec4f86 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
@@ -27,8 +27,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
@@ -39,6 +39,7 @@ import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -52,7 +53,6 @@ import java.util.HashMap;
 import java.util.List;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
 import static org.apache.flink.test.util.TestUtils.tryExecute;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -78,7 +78,6 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	private static TestStreamEnvironment env;
 
-
 	@BeforeClass
 	public static void startTestCluster() {
 		Configuration config = new Configuration();
@@ -103,9 +102,9 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testTumblingProcessingTimeWindow() {
-		final int NUM_ELEMENTS = 3000;
+		final int numElements = 3000;
 		FailingSource.reset();
-		
+
 		try {
 			env.setParallelism(PARALLELISM);
 			env.setStreamTimeCharacteristic(timeCharacteristic);
@@ -115,7 +114,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+					.addSource(new FailingSource(numElements, numElements / 3))
 					.rebalance()
 					.keyBy(0)
 					.timeWindow(Time.of(100, MILLISECONDS))
@@ -145,8 +144,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_ELEMENTS, 1)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numElements, 1)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -158,7 +156,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testSlidingProcessingTimeWindow() {
-		final int NUM_ELEMENTS = 3000;
+		final int numElements = 3000;
 		FailingSource.reset();
 
 		try {
@@ -170,7 +168,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+					.addSource(new FailingSource(numElements, numElements / 3))
 					.rebalance()
 					.keyBy(0)
 					.timeWindow(Time.of(150, MILLISECONDS), Time.of(50, MILLISECONDS))
@@ -200,8 +198,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 							}
 						}
 					})
-					.addSink(new ValidatingSink(NUM_ELEMENTS, 3)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numElements, 3)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -213,7 +210,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testAggregatingTumblingProcessingTimeWindow() {
-		final int NUM_ELEMENTS = 3000;
+		final int numElements = 3000;
 		FailingSource.reset();
 
 		try {
@@ -225,8 +222,8 @@ public class WindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
-					.map(new MapFunction<Tuple2<Long,IntType>, Tuple2<Long,IntType>>() {
+					.addSource(new FailingSource(numElements, numElements / 3))
+					.map(new MapFunction<Tuple2<Long, IntType>, Tuple2<Long, IntType>>() {
 						@Override
 						public Tuple2<Long, IntType> map(Tuple2<Long, IntType> value) {
 							value.f1.value = 1;
@@ -245,8 +242,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 							return new Tuple2<>(a.f0, new IntType(1));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_ELEMENTS, 1)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numElements, 1)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -258,7 +254,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	@Test
 	public void testAggregatingSlidingProcessingTimeWindow() {
-		final int NUM_ELEMENTS = 3000;
+		final int numElements = 3000;
 		FailingSource.reset();
 
 		try {
@@ -270,8 +266,8 @@ public class WindowCheckpointingITCase extends TestLogger {
 			env.getConfig().disableSysoutLogging();
 
 			env
-					.addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
-					.map(new MapFunction<Tuple2<Long,IntType>, Tuple2<Long,IntType>>() {
+					.addSource(new FailingSource(numElements, numElements / 3))
+					.map(new MapFunction<Tuple2<Long, IntType>, Tuple2<Long, IntType>>() {
 						@Override
 						public Tuple2<Long, IntType> map(Tuple2<Long, IntType> value) {
 							value.f1.value = 1;
@@ -289,8 +285,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 							return new Tuple2<>(a.f0, new IntType(1));
 						}
 					})
-					.addSink(new ValidatingSink(NUM_ELEMENTS, 3)).setParallelism(1);
-
+					.addSink(new ValidatingSink(numElements, 3)).setParallelism(1);
 
 			tryExecute(env, "Tumbling Window Test");
 		}
@@ -305,8 +300,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 	// ------------------------------------------------------------------------
 
 	private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
-			implements ListCheckpointed<Integer>, CheckpointListener
-	{
+			implements ListCheckpointed<Integer>, CheckpointListener {
 		private static volatile boolean failedBefore = false;
 
 		private final int numElementsToEmit;
@@ -470,12 +464,17 @@ public class WindowCheckpointingITCase extends TestLogger {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
+	/**
+	 * POJO with int value.
+	 */
 	public static class IntType {
 
 		public int value;
 
 		public IntType() {}
 
-		public IntType(int value) { this.value = value; }
+		public IntType(int value) {
+			this.value = value;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
index e4004c7..21be7ba 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
@@ -18,21 +18,14 @@
 
 package org.apache.flink.test.checkpointing.utils;
 
-import java.io.File;
-import java.net.URI;
-import java.net.URL;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.io.FileUtils;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
@@ -42,12 +35,21 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.util.TestBaseUtils;
+
+import org.apache.commons.io.FileUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URL;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import scala.Option;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
@@ -57,6 +59,9 @@ import scala.concurrent.duration.FiniteDuration;
 
 import static junit.framework.Assert.fail;
 
+/**
+ * Test savepoint migration.
+ */
 public class SavepointMigrationTestBase extends TestBaseUtils {
 
 	@Rule
@@ -120,7 +125,6 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		// Submit the job
 		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
 
-
 		JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph);
 
 		LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID());

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
index 4221670..da6e035 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
@@ -44,6 +44,7 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.Collector;
+
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -133,11 +134,10 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
 	}
 
-
 	@Test
 	public void testSavepointRestoreFromFlink11() throws Exception {
 
-		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
+		final int expectedSuccessfulChecks = 21;
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -165,13 +165,13 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		restoreAndExecute(
 				env,
 				getResourceFilename("stateful-udf-migration-itcase-flink1.1-savepoint"),
-				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
+				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, expectedSuccessfulChecks));
 	}
 
 	@Test
 	public void testSavepointRestoreFromFlink11FromRocksDB() throws Exception {
 
-		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
+		final int expectedSuccessfulChecks = 21;
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
@@ -199,13 +199,13 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		restoreAndExecute(
 				env,
 				getResourceFilename("stateful-udf-migration-itcase-flink1.1-rocksdb-savepoint"),
-				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
+				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, expectedSuccessfulChecks));
 	}
 
 	private static class LegacyCheckpointedSource
 			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
 
-		public static String CHECKPOINTED_STRING = "Here be dragons!";
+		public static String checkpointedString = "Here be dragons!";
 
 		private static final long serialVersionUID = 1L;
 
@@ -237,12 +237,12 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 
 		@Override
 		public void restoreState(String state) throws Exception {
-			assertEquals(CHECKPOINTED_STRING, state);
+			assertEquals(checkpointedString, state);
 		}
 
 		@Override
 		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_STRING;
+			return checkpointedString;
 		}
 	}
 
@@ -271,7 +271,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 
 		@Override
 		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
+			assertEquals(LegacyCheckpointedSource.checkpointedString, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
 
 			synchronized (ctx.getCheckpointLock()) {
@@ -296,12 +296,12 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements Checkpointed<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+		public static Tuple2<String, Long> checkpointedTuple =
 				new Tuple2<>("hello", 42L);
 
 		@Override
@@ -315,11 +315,11 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 
 		@Override
 		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
+			return checkpointedTuple;
 		}
 	}
 
-	public static class RestoringCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class RestoringCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements CheckpointedRestoring<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
@@ -337,7 +337,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
 			out.collect(value);
 
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
 
 		}
@@ -348,13 +348,13 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class LegacyCheckpointedFlatMapWithKeyedState
+	private static class LegacyCheckpointedFlatMapWithKeyedState
 			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements Checkpointed<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+		public static Tuple2<String, Long> checkpointedTuple =
 				new Tuple2<>("hello", 42L);
 
 		private final ValueStateDescriptor<Long> stateDescriptor =
@@ -373,11 +373,11 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 
 		@Override
 		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
+			return checkpointedTuple;
 		}
 	}
 
-	public static class RestoringCheckingFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+	private static class RestoringCheckingFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
 			implements CheckpointedRestoring<Tuple2<String, Long>> {
 
 		private static final long serialVersionUID = 1L;
@@ -404,7 +404,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 			}
 
 			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
 		}
 
@@ -414,7 +414,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -429,7 +429,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class KeyedStateCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+	private static class KeyedStateCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -457,7 +457,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class CheckpointedUdfOperator
+	private static class CheckpointedUdfOperator
 			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
@@ -488,7 +488,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 //					checkpointId,
 //					timestamp);
 //
-//			out.writeUTF(CHECKPOINTED_STRING);
+//			out.writeUTF(checkpointedString);
 //
 //			result.setOperatorState(out.closeAndGetHandle());
 //
@@ -496,7 +496,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 //		}
 	}
 
-	public static class RestoringCheckingUdfOperator
+	private static class RestoringCheckingUdfOperator
 			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
 			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 		private static final long serialVersionUID = 1L;
@@ -535,7 +535,7 @@ public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigratio
 		}
 	}
 
-	public static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
+	private static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
 		private static final long serialVersionUID = 1L;
 
 		private final String accumulatorName;


[08/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java
new file mode 100644
index 0000000..2917441
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java
@@ -0,0 +1,1633 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.functions.RichGroupReduceFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.optimizer.Optimizer;
+import org.apache.flink.test.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CrazyNested;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.FromTupleWithCTor;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.operators.util.CollectionDataSets.PojoContainingTupleAndWritable;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.joda.time.DateTime;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+
+import scala.math.BigInt;
+
+/**
+ * Integration tests for {@link GroupReduceFunction}, {@link RichGroupReduceFunction},
+ * and {@link GroupCombineFunction}.
+ */
+@SuppressWarnings({"serial", "unchecked", "UnusedDeclaration"})
+@RunWith(Parameterized.class)
+public class GroupReduceITCase extends MultipleProgramsTestBase {
+
+	public GroupReduceITCase(TestExecutionMode mode){
+		super(mode);
+	}
+
+	@Test
+	public void testCorrectnessofGroupReduceOnTupleContainingPrimitiveByteArrayWithKeyFieldSelectors() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<byte[], Integer>> ds = CollectionDataSets.getTuple2WithByteArrayDataSet(env);
+		DataSet<Integer> reduceDs = ds.
+				groupBy(0).reduceGroup(new ByteArrayGroupReduce());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "0\n"
+				+ "1\n"
+				+ "2\n"
+				+ "3\n"
+				+ "4\n";
+
+		compareResultAsText(result, expected);
+
+	}
+
+	private static class ByteArrayGroupReduce implements GroupReduceFunction<Tuple2<byte[], Integer>, Integer> {
+		@Override
+		public void reduce(Iterable<Tuple2<byte[], Integer>> values, Collector<Integer> out) throws Exception {
+			int sum = 0;
+			for (Tuple2<byte[], Integer> value : values) {
+				sum += value.f1;
+			}
+			out.collect(sum);
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelector() throws Exception{
+		/*
+		 * check correctness of groupReduce on tuples with key field selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> reduceDs = ds.
+				groupBy(1).reduceGroup(new Tuple3GroupReduce());
+
+		List<Tuple2<Integer, Long>> result = reduceDs.collect();
+
+		String expected = "1,1\n" +
+				"5,2\n" +
+				"15,3\n" +
+				"34,4\n" +
+				"65,5\n" +
+				"111,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithMultipleKeyFieldSelectors() throws Exception {
+		/*
+		 * check correctness of groupReduce on tuples with multiple key field selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
+				groupBy(4, 0).reduceGroup(new Tuple5GroupReduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
+
+		String expected = "1,1,0,P-),1\n" +
+				"2,3,0,P-),1\n" +
+				"2,2,0,P-),2\n" +
+				"3,9,0,P-),2\n" +
+				"3,6,0,P-),3\n" +
+				"4,17,0,P-),1\n" +
+				"4,17,0,P-),2\n" +
+				"5,11,0,P-),1\n" +
+				"5,29,0,P-),2\n" +
+				"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithKeyFieldSelectorAndGroupSorting() throws Exception {
+		/*
+		 * check correctness of groupReduce on tuples with key field selector and group sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).sortGroup(2, Order.ASCENDING).reduceGroup(new Tuple3SortedGroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n"
+				+
+				"5,2,Hello-Hello world\n" +
+				"15,3,Hello world, how are you?-I am fine.-Luke Skywalker\n" +
+				"34,4,Comment#1-Comment#2-Comment#3-Comment#4\n" +
+				"65,5,Comment#5-Comment#6-Comment#7-Comment#8-Comment#9\n" +
+				"111,6,Comment#10-Comment#11-Comment#12-Comment#13-Comment#14-Comment#15\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithKeyExtractor() throws Exception {
+		/*
+		 * check correctness of groupReduce on tuples with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> reduceDs = ds.
+				groupBy(new KeySelector1()).reduceGroup(new Tuple3GroupReduce());
+
+		List<Tuple2<Integer, Long>> result = reduceDs.collect();
+
+		String expected = "1,1\n" +
+				"5,2\n" +
+				"15,3\n" +
+				"34,4\n" +
+				"65,5\n" +
+				"111,6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector1 implements KeySelector<Tuple3<Integer, Long, String>, Long> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Long getKey(Tuple3<Integer, Long, String> in) {
+			return in.f1;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnCustomTypeWithTypeExtractor() throws Exception {
+		/*
+		 * check correctness of groupReduce on custom type with type extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds.
+				groupBy(new KeySelector2()).reduceGroup(new CustomTypeGroupReduce());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "1,0,Hello!\n" +
+				"2,3,Hello!\n" +
+				"3,12,Hello!\n" +
+				"4,30,Hello!\n" +
+				"5,60,Hello!\n" +
+				"6,105,Hello!\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfAllGroupReduceForTuples() throws Exception {
+		/*
+		 * check correctness of all-groupreduce for tuples
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.reduceGroup(new AllAddingTuple3GroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "231,91,Hello World\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfAllGroupReduceForCustomTypes() throws Exception {
+		/*
+		 * check correctness of all-groupreduce for custom types
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds.reduceGroup(new AllAddingCustomTypeGroupReduce());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "91,210,Hello!";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceWithBroadcastSet() throws Exception {
+		/*
+		 * check correctness of groupReduce with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).reduceGroup(new BCTuple3GroupReduce()).withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,55\n" +
+				"5,2,55\n" +
+				"15,3,55\n" +
+				"34,4,55\n" +
+				"65,5,55\n" +
+				"111,6,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceIfUDFReturnsInputObjectsMultipleTimesWhileChangingThem() throws Exception{
+		/*
+		 * check correctness of groupReduce if UDF returns input objects multiple times and changes it in between
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).reduceGroup(new InputReturningTuple3GroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "11,1,Hi!\n" +
+				"21,1,Hi again!\n" +
+				"12,2,Hi!\n" +
+				"22,2,Hi again!\n" +
+				"13,2,Hi!\n" +
+				"23,2,Hi again!\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnCustomTypeWithKeyExtractorAndCombine() throws Exception {
+		/*
+		 * check correctness of groupReduce on custom type with key extractor and combine
+		 */
+		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds.
+				groupBy(new KeySelector3()).reduceGroup(new CustomTypeGroupReduceWithCombine());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "1,0,test1\n" +
+				"2,3,test2\n" +
+				"3,12,test3\n" +
+				"4,30,test4\n" +
+				"5,60,test5\n" +
+				"6,105,test6\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class KeySelector3 implements KeySelector<CustomType, Integer> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Integer getKey(CustomType in) {
+			return in.myInt;
+		}
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithCombine() throws Exception {
+		/*
+		 * check correctness of groupReduce on tuples with combine
+		 */
+		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(2); // important because it determines how often the combiner is called
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, String>> reduceDs = ds.
+				groupBy(1).reduceGroup(new Tuple3GroupReduceWithCombine());
+
+		List<Tuple2<Integer, String>> result = reduceDs.collect();
+
+		String expected = "1,test1\n" +
+				"5,test2\n" +
+				"15,test3\n" +
+				"34,test4\n" +
+				"65,test5\n" +
+				"111,test6\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfAllGroupReduceForTuplesWithCombine() throws Exception {
+		/*
+		 * check correctness of all-groupreduce for tuples with combine
+		 */
+		org.junit.Assume.assumeTrue(mode != TestExecutionMode.COLLECTION);
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env)
+				.map(new IdentityMapper<Tuple3<Integer, Long, String>>()).setParallelism(4);
+
+		Configuration cfg = new Configuration();
+		cfg.setString(Optimizer.HINT_SHIP_STRATEGY, Optimizer.HINT_SHIP_STRATEGY_REPARTITION);
+		DataSet<Tuple2<Integer, String>> reduceDs = ds.reduceGroup(new Tuple3AllGroupReduceWithCombine())
+				.withParameters(cfg);
+
+		List<Tuple2<Integer, String>> result = reduceDs.collect();
+
+		String expected = "322," +
+				"testtesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttesttest\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupreduceWithDescendingGroupSort() throws Exception {
+		/*
+		 * check correctness of groupReduce with descending group sort
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).sortGroup(2, Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n"
+				+
+				"5,2,Hello world-Hello\n" +
+				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
+				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
+				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
+				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testCorrectnessOfGroupReduceOnTuplesWithTupleReturningKeySelector() throws Exception {
+		/*
+		 * check correctness of groupReduce on tuples with tuple-returning key selector
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds.
+				groupBy(new KeySelector4()).reduceGroup(new Tuple5GroupReduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
+
+		String expected = "1,1,0,P-),1\n" +
+				"2,3,0,P-),1\n" +
+				"2,2,0,P-),2\n" +
+				"3,9,0,P-),2\n" +
+				"3,6,0,P-),3\n" +
+				"4,17,0,P-),1\n" +
+				"4,17,0,P-),2\n" +
+				"5,11,0,P-),1\n" +
+				"5,29,0,P-),2\n" +
+				"5,25,0,P-),3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testInputOfCombinerIsSortedForCombinableGroupReduceWithGroupSorting() throws Exception {
+		/*
+		 * check that input of combiner is also sorted for combinable groupReduce with group sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).sortGroup(0, Order.ASCENDING).reduceGroup(new OrderCheckingCombinableReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"4,3,Hello world, how are you?\n" +
+				"7,4,Comment#1\n" +
+				"11,5,Comment#5\n" +
+				"16,6,Comment#10\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testDeepNesting() throws Exception {
+		/*
+		 * Deep nesting test
+		 * + null value in pojo
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CrazyNested> ds = CollectionDataSets.getCrazyNestedDataSet(env);
+		DataSet<Tuple2<String, Integer>> reduceDs = ds.groupBy("nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal")
+				.reduceGroup(new GroupReducer1());
+
+		List<Tuple2<String, Integer>> result = reduceDs.collect();
+
+		String expected = "aa,1\nbb,2\ncc,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class GroupReducer1 implements GroupReduceFunction<CollectionDataSets.CrazyNested, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<CrazyNested> values,
+				Collector<Tuple2<String, Integer>> out)
+						throws Exception {
+			int c = 0; String n = null;
+			for (CrazyNested v : values) {
+				c++; // haha
+				n = v.nestLvl1.nestLvl2.nestLvl3.nestLvl4.f1nal;
+			}
+			out.collect(new Tuple2<>(n, c));
+		}
+	}
+
+	@Test
+	public void testPojoExtendingFromTupleWithCustomFields() throws Exception {
+		/*
+		 * Test Pojo extending from tuple WITH custom fields
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<FromTupleWithCTor> ds = CollectionDataSets.getPojoExtendingFromTuple(env);
+		DataSet<Integer> reduceDs = ds.groupBy("special", "f2")
+				.reduceGroup(new GroupReducer2());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "3\n2\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer2 implements GroupReduceFunction<FromTupleWithCTor, Integer> {
+
+		@Override
+		public void reduce(Iterable<FromTupleWithCTor> values, Collector<Integer> out) {
+			out.collect(countElements(values));
+		}
+	}
+
+	@Test
+	public void testPojoContainigWritableAndTuples() throws Exception {
+		/*
+		 * Test Pojo containing a Writable and Tuples
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<PojoContainingTupleAndWritable> ds = CollectionDataSets.getPojoContainingTupleAndWritable(env);
+		DataSet<Integer> reduceDs = ds.groupBy("hadoopFan", "theTuple.*") // full tuple selection
+				.reduceGroup(new GroupReducer3());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "1\n5\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer3 implements GroupReduceFunction<PojoContainingTupleAndWritable, Integer> {
+
+		@Override
+		public void reduce(Iterable<PojoContainingTupleAndWritable> values, Collector<Integer> out) {
+			out.collect(countElements(values));
+		}
+	}
+
+	@Test
+	public void testTupleContainingPojosAndRegularFields() throws Exception {
+		/*
+		 * Test Tuple containing pojos and regular fields
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, CrazyNested, POJO>> ds = CollectionDataSets.getTupleContainingPojos(env);
+
+		DataSet<Integer> reduceDs = ds.groupBy("f0", "f1.*") // nested full tuple selection
+				.reduceGroup(new GroupReducer4());
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "3\n1\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer4 implements GroupReduceFunction<Tuple3<Integer, CrazyNested, POJO>, Integer> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, CrazyNested, POJO>> values, Collector<Integer> out) {
+			out.collect(countElements(values));
+		}
+	}
+
+	@Test
+	public void testStringBasedDefinitionOnGroupSort() throws Exception {
+		/*
+		 * Test string-based definition on group sort, based on test:
+		 * check correctness of groupReduce with descending group sort
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds.
+				groupBy(1).sortGroup("f2", Order.DESCENDING).reduceGroup(new Tuple3SortedGroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n"
+				+
+				"5,2,Hello world-Hello\n" +
+				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
+				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
+				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
+				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testIntBasedDefinitionOnGroupSortForFullNestedTuple() throws Exception {
+		/*
+		 * Test int-based definition on group sort, for (full) nested Tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
+		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup(0, Order.DESCENDING).reduceGroup(new NestedTupleReducer());
+		List<String> result = reduceDs.collect();
+
+		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
+				"b--(2,2)-\n" +
+				"c--(4,9)-(3,6)-(3,3)-\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testIntBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception {
+		/*
+		 * Test int-based definition on group sort, for (partial) nested Tuple ASC
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("f1")
+				.sortGroup("f0.f0", Order.ASCENDING)
+				.sortGroup("f0.f1", Order.ASCENDING)
+				.reduceGroup(new NestedTupleReducer());
+		List<String> result = reduceDs.collect();
+
+		String expected = "a--(1,2)-(1,3)-(2,1)-\n" +
+				"b--(2,2)-\n" +
+				"c--(3,3)-(3,6)-(4,9)-\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testStringBasedDefinitionOnGroupSortForPartialNestedTuple() throws Exception {
+		/*
+		 * Test string-based definition on group sort, for (partial) nested Tuple DESC
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).reduceGroup(new NestedTupleReducer());
+		List<String> result = reduceDs.collect();
+
+		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
+				"b--(2,2)-\n" +
+				"c--(4,9)-(3,3)-(3,6)-\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeys() throws Exception {
+		/*
+		 * Test string-based definition on group sort, for two grouping keys
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds = CollectionDataSets.getGroupSortedNestedTupleDataSet(env);
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("f1").sortGroup("f0.f0", Order.DESCENDING).sortGroup("f0.f1", Order.DESCENDING).reduceGroup(new NestedTupleReducer());
+		List<String> result = reduceDs.collect();
+
+		String expected = "a--(2,1)-(1,3)-(1,2)-\n" +
+				"b--(2,2)-\n" +
+				"c--(4,9)-(3,6)-(3,3)-\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testStringBasedDefinitionOnGroupSortForTwoGroupingKeysWithPojos() throws Exception {
+		/*
+		 * Test string-based definition on group sort, for two grouping keys with Pojos
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<PojoContainingTupleAndWritable> ds = CollectionDataSets.getGroupSortedPojoContainingTupleAndWritable(env);
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("hadoopFan").sortGroup("theTuple.f0", Order.DESCENDING).sortGroup("theTuple.f1", Order.DESCENDING)
+				.reduceGroup(new GroupReducer5());
+		List<String> result = reduceDs.collect();
+
+		String expected = "1---(10,100)-\n"
+				+
+				"2---(30,600)-(30,400)-(30,200)-(20,201)-(20,200)-\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testTupleKeySelectorGroupSort() throws Exception {
+		/*
+		 * check correctness of sorted groupReduce on tuples with keyselector sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> reduceDs = ds
+				.groupBy(new LongFieldExtractor<Tuple3<Integer, Long, String>>(1))
+				.sortGroup(new StringFieldExtractor<Tuple3<Integer, Long, String>>(2), Order.DESCENDING)
+				.reduceGroup(new Tuple3SortedGroupReduce());
+
+		List<Tuple3<Integer, Long, String>> result = reduceDs.collect();
+
+		String expected = "1,1,Hi\n"
+				+
+				"5,2,Hello world-Hello\n" +
+				"15,3,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
+				"34,4,Comment#4-Comment#3-Comment#2-Comment#1\n" +
+				"65,5,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
+				"111,6,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class TwoTuplePojoExtractor implements KeySelector<CustomType, Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> getKey(CustomType value) throws Exception {
+			return new Tuple2<>(value.myInt, value.myInt);
+		}
+	}
+
+	private static class StringPojoExtractor implements KeySelector<CustomType, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(CustomType value) throws Exception {
+			return value.myString;
+		}
+	}
+
+	@Test
+	public void testPojoKeySelectorGroupSort() throws Exception {
+		/*
+		 * check correctness of sorted groupReduce on custom type with keyselector sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> reduceDs = ds
+				.groupBy(new TwoTuplePojoExtractor())
+				.sortGroup(new StringPojoExtractor(), Order.DESCENDING)
+				.reduceGroup(new CustomTypeSortedGroupReduce());
+
+		List<CustomType> result = reduceDs.collect();
+
+		String expected = "1,0,Hi\n"
+				+
+				"2,3,Hello world-Hello\n" +
+				"3,12,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
+				"4,30,Comment#4-Comment#3-Comment#2-Comment#1\n" +
+				"5,60,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
+				"6,105,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class LongFieldExtractor<T extends Tuple>  implements KeySelector<T, Long> {
+		private static final long serialVersionUID = 1L;
+		private int field;
+
+		public LongFieldExtractor() { }
+
+		public LongFieldExtractor(int field) {
+			this.field = field;
+		}
+
+		@Override
+		public Long getKey(T t) throws Exception {
+			return ((Tuple) t).getField(field);
+		}
+	}
+
+	private static class IntFieldExtractor<T extends Tuple>  implements KeySelector<T, Integer> {
+		private static final long serialVersionUID = 1L;
+		private int field;
+
+		public IntFieldExtractor() { }
+
+		public IntFieldExtractor(int field) {
+			this.field = field;
+		}
+
+		@Override
+		public Integer getKey(T t) throws Exception {
+			return ((Tuple) t).getField(field);
+		}
+	}
+
+	private static class StringFieldExtractor<T extends Tuple>  implements KeySelector<T, String> {
+		private static final long serialVersionUID = 1L;
+		private int field;
+
+		public StringFieldExtractor() { }
+
+		public StringFieldExtractor(int field) {
+			this.field = field;
+		}
+
+		@Override
+		public String getKey(T t) throws Exception {
+			return t.getField(field);
+		}
+	}
+
+	@Test
+	public void testTupleKeySelectorSortWithCombine() throws Exception {
+		/*
+		 * check correctness of sorted groupReduce with combine on tuples with keyselector sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<Integer, String>> reduceDs = ds.
+				groupBy(new LongFieldExtractor<Tuple3<Integer, Long, String>>(1))
+				.sortGroup(new StringFieldExtractor<Tuple3<Integer, Long, String>>(2), Order.DESCENDING)
+				.reduceGroup(new Tuple3SortedGroupReduceWithCombine());
+
+		List<Tuple2<Integer, String>> result = reduceDs.collect();
+
+		if (super.mode != TestExecutionMode.COLLECTION) {
+			String expected = "1,Hi\n"
+					+
+					"5,Hello world-Hello\n" +
+					"15,Luke Skywalker-I am fine.-Hello world, how are you?\n" +
+					"34,Comment#4-Comment#3-Comment#2-Comment#1\n" +
+					"65,Comment#9-Comment#8-Comment#7-Comment#6-Comment#5\n" +
+					"111,Comment#15-Comment#14-Comment#13-Comment#12-Comment#11-Comment#10\n";
+
+			compareResultAsTuples(result, expected);
+		}
+	}
+
+	private static class FiveToTwoTupleExtractor implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Long, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, Integer, String, Long> in) {
+			return new Tuple2<>(in.f4, in.f2);
+		}
+	}
+
+	@Test
+	public void testTupleKeySelectorSortCombineOnTuple() throws Exception {
+		/*
+		 * check correctness of sorted groupReduceon with Tuple2 keyselector sorting
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> reduceDs = ds
+				.groupBy(new IntFieldExtractor<Tuple5<Integer, Long, Integer, String, Long>>(0))
+				.sortGroup(new FiveToTwoTupleExtractor(), Order.DESCENDING)
+				.reduceGroup(new Tuple5SortedGroupReduce());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = reduceDs.collect();
+
+		String expected = "1,1,0,Hallo,1\n"
+				+
+				"2,5,0,Hallo Welt-Hallo Welt wie,1\n" +
+				"3,15,0,BCD-ABC-Hallo Welt wie gehts?,2\n" +
+				"4,34,0,FGH-CDE-EFG-DEF,1\n" +
+				"5,65,0,IJK-HIJ-KLM-JKL-GHI,1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class GroupReducer5 implements GroupReduceFunction<CollectionDataSets.PojoContainingTupleAndWritable, String> {
+		@Override
+		public void reduce(
+				Iterable<PojoContainingTupleAndWritable> values,
+				Collector<String> out) throws Exception {
+			boolean once = false;
+			StringBuilder concat = new StringBuilder();
+			for (PojoContainingTupleAndWritable value : values) {
+				if (!once) {
+					concat.append(value.hadoopFan.get());
+					concat.append("---");
+					once = true;
+				}
+				concat.append(value.theTuple);
+				concat.append("-");
+			}
+			out.collect(concat.toString());
+		}
+	}
+
+	@Test
+	public void testGroupingWithPojoContainingMultiplePojos() throws Exception {
+		/*
+		 * Test grouping with pojo containing multiple pojos (was a bug)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<CollectionDataSets.PojoWithMultiplePojos> ds = CollectionDataSets.getPojoWithMultiplePojos(env);
+
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("p2.a2")
+				.reduceGroup(new GroupReducer6());
+		List<String> result = reduceDs.collect();
+
+		String expected = "b\nccc\nee\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer6 implements GroupReduceFunction<CollectionDataSets.PojoWithMultiplePojos, String> {
+		@Override
+		public void reduce(
+				Iterable<CollectionDataSets.PojoWithMultiplePojos> values,
+				Collector<String> out) throws Exception {
+			StringBuilder concat = new StringBuilder();
+			for (CollectionDataSets.PojoWithMultiplePojos value : values) {
+				concat.append(value.p2.a2);
+			}
+			out.collect(concat.toString());
+		}
+	}
+
+	@Test
+	public void testJavaCollectionsWithinPojos() throws Exception {
+		/*
+		 * Test Java collections within pojos ( == test kryo)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("key")
+				.reduceGroup(new GroupReducer7());
+		List<String> result = reduceDs.collect();
+
+		String expected = "callFor key 0 we got: pojo.a=apojo.a=bFor key 0 we got: pojo.a=a2pojo.a=b2\n";
+
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer7 implements GroupReduceFunction<CollectionDataSets.PojoWithCollection, String> {
+
+		@Override
+		public void reduce(Iterable<CollectionDataSets.PojoWithCollection> values, Collector<String> out) {
+			StringBuilder concat = new StringBuilder();
+			concat.append("call");
+			for (CollectionDataSets.PojoWithCollection value : values) {
+				concat.append("For key ").append(value.key).append(" we got: ");
+
+				for (CollectionDataSets.Pojo1 p :value.pojos) {
+					concat.append("pojo.a=").append(p.a);
+				}
+			}
+			out.collect(concat.toString());
+		}
+	}
+
+	@Test
+	public void testGroupByGenericType() throws Exception {
+		/*
+		 * Group by generic type
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
+
+		DataSet<CollectionDataSets.PojoWithCollection> ds = CollectionDataSets.getPojoWithCollection(env);
+
+		// f0.f0 is first integer
+		DataSet<String> reduceDs = ds.groupBy("bigInt")
+				.reduceGroup(new GroupReducer8());
+		List<String> result = reduceDs.collect();
+		ExecutionConfig ec = env.getConfig();
+
+		// check if automatic type registration with Kryo worked
+		Assert.assertTrue(ec.getRegisteredKryoTypes().contains(BigInt.class));
+		Assert.assertFalse(ec.getRegisteredKryoTypes().contains(java.sql.Date.class));
+
+		String expected = null;
+
+		String localExpected = "[call\n" +
+				"For key 92233720368547758070 we got:\n" +
+				"PojoWithCollection{pojos.size()=2, key=0, sqlDate=2033-05-18, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=10, mixed=[{someKey=1}, /this/is/wrong, uhlala]}\n" +
+				"For key 92233720368547758070 we got:\n" +
+				"PojoWithCollection{pojos.size()=2, key=0, sqlDate=1976-05-03, bigInt=92233720368547758070, bigDecimalKeepItNull=null, scalaBigInt=31104000, mixed=null}]";
+
+		Assert.assertEquals(localExpected, result.toString());
+	}
+
+	@Test
+	public void testGroupReduceSelectorKeysWithSemProps() throws Exception {
+
+		/*
+		 * Test that semantic properties are correctly adapted when using Selector Keys
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<Integer, Long>> reduceDs = ds
+				// group by selector key
+				.groupBy(new KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Long>() {
+					@Override
+					public Long getKey(Tuple5<Integer, Long, Integer, String, Long> v) throws Exception {
+						return (v.f0 * v.f1) - (v.f2 * v.f4);
+					}
+				})
+				.reduceGroup(
+						new GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>>() {
+							@Override
+							public void reduce(Iterable<Tuple5<Integer, Long, Integer, String, Long>> values, Collector<Tuple5<Integer, Long, Integer, String, Long>> out) throws Exception {
+								for (Tuple5<Integer, Long, Integer, String, Long> v : values) {
+									out.collect(v);
+								}
+							}
+						})
+						// add forward field information
+						.withForwardedFields("0")
+						// group again and reduce
+						.groupBy(0).reduceGroup(
+								new GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>>() {
+									@Override
+									public void reduce(Iterable<Tuple5<Integer, Long, Integer, String, Long>> values, Collector<Tuple2<Integer, Long>> out) throws Exception {
+										int k = 0;
+										long s = 0;
+										for (Tuple5<Integer, Long, Integer, String, Long> v : values) {
+											k = v.f0;
+											s += v.f1;
+										}
+										out.collect(new Tuple2<>(k, s));
+									}
+								}
+								);
+
+		List<Tuple2<Integer, Long>> result = reduceDs.collect();
+
+		String expected = "1,1\n" +
+				"2,5\n" +
+				"3,15\n" +
+				"4,34\n" +
+				"5,65\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testGroupReduceWithAtomicValue() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Integer> ds = env.fromElements(1, 1, 2, 3, 4);
+		DataSet<Integer> reduceDs = ds.groupBy("*").reduceGroup(new GroupReduceFunction<Integer, Integer>() {
+			@Override
+			public void reduce(Iterable<Integer> values, Collector<Integer> out) throws Exception {
+				out.collect(values.iterator().next());
+			}
+		});
+
+		List<Integer> result = reduceDs.collect();
+
+		String expected = "1\n" +
+				"2\n" +
+				"3\n" +
+				"4";
+
+		compareResultAsText(result, expected);
+	}
+
+	/**
+	 * Fix for FLINK-2019.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testJodatimeDateTimeWithKryo() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		DataSet<Tuple2<Integer, DateTime>> ds = env.fromElements(new Tuple2<>(1, DateTime.now()));
+		DataSet<Tuple2<Integer, DateTime>> reduceDs = ds.groupBy("f1").sum(0).project(0);
+
+		List<Tuple2<Integer, DateTime>> result = reduceDs.collect();
+
+		String expected = "1\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	/**
+	 * Fix for FLINK-2158.
+	 *
+	 * @throws Exception
+	 */
+	@Test
+	public void testDateNullException() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Integer, Date>> in = env.fromElements(new Tuple2<>(0, new Date(1230000000)),
+				new Tuple2<Integer, Date>(1, null),
+				new Tuple2<>(2, new Date(1230000000))
+		);
+
+		DataSet<String> r = in.groupBy(0).reduceGroup(new GroupReduceFunction<Tuple2<Integer, Date>, String>() {
+			@Override
+			public void reduce(Iterable<Tuple2<Integer, Date>> values, Collector<String> out) throws Exception {
+				for (Tuple2<Integer, Date> e : values) {
+					out.collect(Integer.toString(e.f0));
+				}
+			}
+		});
+
+		List<String> result = r.collect();
+
+		String expected = "0\n1\n2\n";
+		compareResultAsText(result, expected);
+	}
+
+	private static class GroupReducer8 implements GroupReduceFunction<CollectionDataSets.PojoWithCollection, String> {
+		@Override
+		public void reduce(Iterable<CollectionDataSets.PojoWithCollection> values, Collector<String> out) {
+			StringBuilder concat = new StringBuilder();
+			concat.append("call");
+			for (CollectionDataSets.PojoWithCollection value : values) {
+				concat.append("\nFor key ").append(value.bigInt).append(" we got:\n").append(value);
+			}
+			out.collect(concat.toString());
+		}
+	}
+
+	private static class NestedTupleReducer implements GroupReduceFunction<Tuple2<Tuple2<Integer, Integer>, String>, String> {
+		@Override
+		public void reduce(Iterable<Tuple2<Tuple2<Integer, Integer>, String>> values, Collector<String> out) {
+			boolean once = false;
+			StringBuilder concat = new StringBuilder();
+			for (Tuple2<Tuple2<Integer, Integer>, String> value : values) {
+				if (!once) {
+					concat.append(value.f1).append("--");
+					once = true;
+				}
+				concat.append(value.f0); // the tuple with the sorted groups
+				concat.append("-");
+			}
+			out.collect(concat.toString());
+		}
+	}
+
+	private static class Tuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, Long>> out) {
+			int i = 0;
+			long l = 0L;
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0;
+				l = t.f1;
+			}
+
+			out.collect(new Tuple2<>(i, l));
+
+		}
+	}
+
+	private static class Tuple3SortedGroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+			int sum = 0;
+			long key = 0;
+			StringBuilder concat = new StringBuilder();
+
+			for (Tuple3<Integer, Long, String> next : values) {
+				sum += next.f0;
+				key = next.f1;
+				concat.append(next.f2).append("-");
+			}
+
+			if (concat.length() > 0) {
+				concat.setLength(concat.length() - 1);
+			}
+
+			out.collect(new Tuple3<>(sum, key, concat.toString()));
+		}
+	}
+
+	private static class Tuple5GroupReduce implements GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> values,
+				Collector<Tuple5<Integer, Long, Integer, String, Long>> out) {
+			int i = 0;
+			long l = 0L;
+			long l2 = 0L;
+
+			for (Tuple5<Integer, Long, Integer, String, Long> t : values) {
+				i = t.f0;
+				l += t.f1;
+				l2 = t.f4;
+			}
+
+			out.collect(new Tuple5<>(i, l, 0, "P-)", l2));
+		}
+	}
+
+	private static class Tuple5SortedGroupReduce implements GroupReduceFunction<Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(
+				Iterable<Tuple5<Integer, Long, Integer, String, Long>> values,
+				Collector<Tuple5<Integer, Long, Integer, String, Long>> out) {
+			int i = 0;
+			long l = 0L;
+			long l2 = 0L;
+			StringBuilder concat = new StringBuilder();
+
+			for (Tuple5<Integer, Long, Integer, String, Long> t : values) {
+				i = t.f0;
+				l += t.f1;
+				concat.append(t.f3).append("-");
+				l2 = t.f4;
+			}
+			if (concat.length() > 0) {
+				concat.setLength(concat.length() - 1);
+			}
+
+			out.collect(new Tuple5<>(i, l, 0, concat.toString(), l2));
+		}
+	}
+
+	private static class CustomTypeGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
+			final Iterator<CustomType> iter = values.iterator();
+
+			CustomType o = new CustomType();
+			CustomType c = iter.next();
+
+			o.myString = "Hello!";
+			o.myInt = c.myInt;
+			o.myLong = c.myLong;
+
+			while (iter.hasNext()) {
+				CustomType next = iter.next();
+				o.myLong += next.myLong;
+			}
+
+			out.collect(o);
+
+		}
+	}
+
+	private static class CustomTypeSortedGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
+			final Iterator<CustomType> iter = values.iterator();
+
+			CustomType o = new CustomType();
+			CustomType c = iter.next();
+
+			StringBuilder concat = new StringBuilder(c.myString);
+			o.myInt = c.myInt;
+			o.myLong = c.myLong;
+
+			while (iter.hasNext()) {
+				CustomType next = iter.next();
+				concat.append("-").append(next.myString);
+				o.myLong += next.myLong;
+
+			}
+
+			o.myString = concat.toString();
+			out.collect(o);
+
+		}
+	}
+
+	private static class InputReturningTuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			for (Tuple3<Integer, Long, String> t : values) {
+
+				if (t.f0 < 4) {
+					t.f2 = "Hi!";
+					t.f0 += 10;
+					out.collect(t);
+					t.f0 += 10;
+					t.f2 = "Hi again!";
+					out.collect(t);
+				}
+			}
+		}
+	}
+
+	private static class AllAddingTuple3GroupReduce implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			int i = 0;
+			long l = 0L;
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0;
+				l += t.f1;
+			}
+
+			out.collect(new Tuple3<>(i, l, "Hello World"));
+		}
+	}
+
+	private static class AllAddingCustomTypeGroupReduce implements GroupReduceFunction<CustomType, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<CustomType> values, Collector<CustomType> out) {
+
+			CustomType o = new CustomType(0, 0, "Hello!");
+
+			for (CustomType next : values) {
+				o.myInt += next.myInt;
+				o.myLong += next.myLong;
+			}
+
+			out.collect(o);
+		}
+	}
+
+	private static class BCTuple3GroupReduce extends RichGroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+		private String f2Replace = "";
+
+		@Override
+		public void open(Configuration config) {
+
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			f2Replace = sum + "";
+
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			int i = 0;
+			long l = 0L;
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0;
+				l = t.f1;
+			}
+
+			out.collect(new Tuple3<>(i, l, this.f2Replace));
+
+		}
+	}
+
+	private static class Tuple3GroupReduceWithCombine
+		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
+					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			Tuple3<Integer, Long, String> o = new Tuple3<>(0, 0L, "");
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				o.f0 += t.f0;
+				o.f1 = t.f1;
+				o.f2 = "test" + o.f1;
+			}
+
+			out.collect(o);
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
+
+			int i = 0;
+			String s = "";
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0;
+				s = t.f2;
+			}
+
+			out.collect(new Tuple2<>(i, s));
+
+		}
+	}
+
+	private static class Tuple3SortedGroupReduceWithCombine
+		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
+					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+			int sum = 0;
+			long key = 0;
+			StringBuilder concat = new StringBuilder();
+
+			for (Tuple3<Integer, Long, String> next : values) {
+				sum += next.f0;
+				key = next.f1;
+				concat.append(next.f2).append("-");
+			}
+
+			if (concat.length() > 0) {
+				concat.setLength(concat.length() - 1);
+			}
+
+			out.collect(new Tuple3<>(sum, key, concat.toString()));
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
+			int i = 0;
+			String s = "";
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0;
+				s = t.f2;
+			}
+
+			out.collect(new Tuple2<>(i, s));
+		}
+	}
+
+	private static class Tuple3AllGroupReduceWithCombine
+		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple2<Integer, String>>,
+					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			Tuple3<Integer, Long, String> o = new Tuple3<>(0, 0L, "");
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				o.f0 += t.f0;
+				o.f1 += t.f1;
+				o.f2 += "test";
+			}
+
+			out.collect(o);
+		}
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple2<Integer, String>> out) {
+
+			int i = 0;
+			String s = "";
+
+			for (Tuple3<Integer, Long, String> t : values) {
+				i += t.f0 + t.f1;
+				s += t.f2;
+			}
+
+			out.collect(new Tuple2<>(i, s));
+
+		}
+	}
+
+	private static class CustomTypeGroupReduceWithCombine
+		implements GroupReduceFunction<CustomType, CustomType>,
+					GroupCombineFunction<CustomType, CustomType> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void combine(Iterable<CustomType> values, Collector<CustomType> out) throws Exception {
+
+			CustomType o = new CustomType();
+
+			for (CustomType c : values) {
+				o.myInt = c.myInt;
+				o.myLong += c.myLong;
+				o.myString = "test" + c.myInt;
+			}
+
+			out.collect(o);
+		}
+
+		@Override
+		public void reduce(Iterable<CustomType> values, Collector<CustomType> out)  {
+
+			CustomType o = new CustomType(0, 0, "");
+
+			for (CustomType c : values) {
+				o.myInt = c.myInt;
+				o.myLong += c.myLong;
+				o.myString = c.myString;
+			}
+
+			out.collect(o);
+
+		}
+	}
+
+	private static class OrderCheckingCombinableReduce
+		implements GroupReduceFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>,
+					GroupCombineFunction<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void reduce(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) throws Exception {
+			Iterator<Tuple3<Integer, Long, String>> it = values.iterator();
+			Tuple3<Integer, Long, String> t = it.next();
+
+			int i = t.f0;
+			out.collect(t);
+
+			while (it.hasNext()) {
+				t = it.next();
+				if (i > t.f0 || t.f2.equals("INVALID-ORDER!")) {
+					t.f2 = "INVALID-ORDER!";
+					out.collect(t);
+				}
+			}
+		}
+
+		@Override
+		public void combine(Iterable<Tuple3<Integer, Long, String>> values, Collector<Tuple3<Integer, Long, String>> out) {
+
+			Iterator<Tuple3<Integer, Long, String>> it = values.iterator();
+			Tuple3<Integer, Long, String> t = it.next();
+
+			int i = t.f0;
+			out.collect(t);
+
+			while (it.hasNext()) {
+				t = it.next();
+				if (i > t.f0) {
+					t.f2 = "INVALID-ORDER!";
+					out.collect(t);
+				}
+			}
+
+		}
+
+	}
+
+	private static final class IdentityMapper<T> extends RichMapFunction<T, T> {
+		@Override
+		public T map(T value) {
+			return value;
+		}
+	}
+
+	private static int countElements(Iterable<?> iterable) {
+		int c = 0;
+		for (@SuppressWarnings("unused") Object o : iterable) {
+			c++;
+		}
+		return c;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
new file mode 100644
index 0000000..38adb62
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java
@@ -0,0 +1,945 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.distributions.DataDistribution;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.utils.DataSetUtils;
+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.operators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets.CustomType;
+import org.apache.flink.test.operators.util.CollectionDataSets.POJO;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Integration tests for {@link JoinFunction} and {@link FlatJoinFunction}.
+ */
+@SuppressWarnings("serial")
+@RunWith(Parameterized.class)
+public class JoinITCase extends MultipleProgramsTestBase {
+
+	public JoinITCase(TestExecutionMode mode) {
+		super(mode);
+	}
+
+	@Test
+	public void testUDFJoinOnTuplesWithKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(1)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testeUDFJoinOnTuplesWithMultipleKeyFieldPositions() throws Exception {
+		/*
+		 * UDF Join on tuples with multiple key field positions
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.join(ds2)
+						.where(0, 1)
+						.equalTo(0, 4)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt wie gehts?\n" +
+				"Hello world,ABC\n" +
+				"I am fine.,HIJ\n" +
+				"I am fine.,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testDefaultJoinOnTuples() throws Exception {
+		/*
+		 * Default Join on tuples
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> joinDs =
+				ds1.join(ds2)
+						.where(0)
+						.equalTo(2);
+
+		List<Tuple2<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>>> result = joinDs.collect();
+
+		String expected = "(1,1,Hi),(2,2,1,Hallo Welt,2)\n" +
+				"(2,2,Hello),(2,3,2,Hallo Welt wie,1)\n" +
+				"(3,2,Hello world),(3,4,3,Hallo Welt wie gehts?,2)\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	@Test
+	public void testJoinWithHuge() throws Exception {
+		/*
+		 * Join with Huge
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs = ds1.joinWithHuge(ds2)
+				.where(1)
+				.equalTo(1)
+				.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithTiny() throws Exception {
+		/*
+		 * Join with Tiny
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.joinWithTiny(ds2)
+						.where(1)
+						.equalTo(1)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinThatReturnsTheLeftInputObject() throws Exception {
+		/*
+		 * Join that returns the left input object
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(1)
+						.with(new LeftReturningJoin());
+
+		List<Tuple3<Integer, Long, String>> result = joinDs.collect();
+
+		String expected = "1,1,Hi\n" +
+				"2,2,Hello\n" +
+				"3,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinThatReturnsTheRightInputObject() throws Exception {
+		/*
+		 * Join that returns the right input object
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(1)
+						.with(new RightReturningJoin());
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> result = joinDs.collect();
+
+		String expected = "1,1,0,Hallo,1\n" +
+				"2,2,1,Hallo Welt,2\n" +
+				"2,2,1,Hallo Welt,2\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithBroadcastSet() throws Exception {
+		/*
+		 * Join with broadcast set
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> intDs = CollectionDataSets.getIntegerDataSet(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.getSmall5TupleDataSet(env);
+		DataSet<Tuple3<String, String, Integer>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(4)
+						.with(new T3T5BCJoin())
+						.withBroadcastSet(intDs, "ints");
+
+		List<Tuple3<String, String, Integer>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo,55\n" +
+				"Hi,Hallo Welt wie,55\n" +
+				"Hello,Hallo Welt,55\n" +
+				"Hello world,Hallo Welt,55\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinOnACustomTypeInputWithKeyExtractorAndATupleInputWithKeyFieldSelector()
+			throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.join(ds2)
+						.where(new KeySelector1())
+						.equalTo(0)
+						.with(new CustT3Join());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hi\n" +
+				"Hello,Hello\n" +
+				"Hello world,Hello\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	private static class KeySelector1 implements KeySelector<CustomType, Integer> {
+		@Override
+		public Integer getKey(CustomType value) {
+			return value.myInt;
+		}
+	}
+
+	@Test
+	public void testProjectOnATuple1Input() throws Exception {
+		/*
+		 * Project join on a tuple input 1
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple6<String, Long, String, Integer, Long, Long>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(1)
+						.projectFirst(2, 1)
+						.projectSecond(3)
+						.projectFirst(0)
+						.projectSecond(4, 1);
+
+		List<Tuple6<String, Long, String, Integer, Long, Long>> result = joinDs.collect();
+
+		String expected = "Hi,1,Hallo,1,1,1\n" +
+				"Hello,2,Hallo Welt,2,2,2\n" +
+				"Hello world,2,Hallo Welt,3,2,2\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testProjectJoinOnATuple2Input() throws Exception {
+		/*
+		 * Project join on a tuple input 2
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple6<String, String, Long, Long, Long, Integer>> joinDs =
+				ds1.join(ds2)
+						.where(1)
+						.equalTo(1)
+						.projectSecond(3)
+						.projectFirst(2, 1)
+						.projectSecond(4, 1)
+						.projectFirst(0);
+
+		List<Tuple6<String, String, Long, Long, Long, Integer>> result = joinDs.collect();
+
+		String expected = "Hallo,Hi,1,1,1,1\n" +
+				"Hallo Welt,Hello,2,2,2,2\n" +
+				"Hallo Welt,Hello world,2,2,2,3\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinOnATupleInputWithKeyFieldSelectorAndACustomTypeInputWithKeyExtractor()
+			throws Exception {
+		/*
+		 * Join on a tuple input with key field selector and a custom type input with key extractor
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.join(ds2)
+						.where(1).equalTo(new KeySelector2())
+						.with(new T3CustJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hello\n" +
+				"Hello,Hello world\n" +
+				"Hello world,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector2 implements KeySelector<CustomType, Long> {
+		@Override
+		public Long getKey(CustomType value) {
+			return value.myLong;
+		}
+	}
+
+	@Test
+	public void testDefaultJoinOnTwoCustomTypeInputsWithKeyExtractors() throws Exception {
+		/*
+		 * (Default) Join on two custom type inputs with key extractors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+
+		DataSet<Tuple2<CustomType, CustomType>> joinDs =
+				ds1.join(ds2)
+						.where(new KeySelector5())
+						.equalTo(new KeySelector6());
+
+		List<Tuple2<CustomType, CustomType>> result = joinDs.collect();
+
+		String expected = "1,0,Hi,1,0,Hi\n" +
+				"2,1,Hello,2,1,Hello\n" +
+				"2,1,Hello,2,2,Hello world\n" +
+				"2,2,Hello world,2,1,Hello\n" +
+				"2,2,Hello world,2,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsClosureCleaner() throws Exception {
+		/*
+		 * (Default) Join on two custom type inputs with key extractors, implemented as inner classes to test closure
+		 * cleaning
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+
+		DataSet<Tuple2<CustomType, CustomType>> joinDs =
+				ds1.join(ds2)
+						.where(new KeySelector<CustomType, Integer>() {
+							@Override
+							public Integer getKey(CustomType value) {
+								return value.myInt;
+							}
+						})
+						.equalTo(new KeySelector<CustomType, Integer>() {
+
+							@Override
+							public Integer getKey(CustomType value) throws Exception {
+								return value.myInt;
+							}
+						});
+
+		List<Tuple2<CustomType, CustomType>> result = joinDs.collect();
+
+		String expected = "1,0,Hi,1,0,Hi\n" +
+				"2,1,Hello,2,1,Hello\n" +
+				"2,1,Hello,2,2,Hello world\n" +
+				"2,2,Hello world,2,1,Hello\n" +
+				"2,2,Hello world,2,2,Hello world\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testDefaultJoinOnTwoCustomTypeInputsWithInnerClassKeyExtractorsDisabledClosureCleaner() throws Exception {
+		/*
+		 * (Default) Join on two custom type inputs with key extractors, check if disableing closure cleaning works
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().disableClosureCleaner();
+
+		DataSet<CustomType> ds1 = CollectionDataSets.getCustomTypeDataSet(env);
+		DataSet<CustomType> ds2 = CollectionDataSets.getSmallCustomTypeDataSet(env);
+		boolean correctExceptionTriggered = false;
+		try {
+			DataSet<Tuple2<CustomType, CustomType>> joinDs =
+					ds1.join(ds2)
+							.where(new KeySelector<CustomType, Integer>() {
+								@Override
+								public Integer getKey(CustomType value) {
+									return value.myInt;
+								}
+							})
+							.equalTo(new KeySelector<CustomType, Integer>() {
+
+								@Override
+								public Integer getKey(CustomType value) throws Exception {
+									return value.myInt;
+								}
+							});
+		} catch (InvalidProgramException ex) {
+			correctExceptionTriggered = (ex.getCause() instanceof java.io.NotSerializableException);
+		}
+		Assert.assertTrue(correctExceptionTriggered);
+	}
+
+	private static class KeySelector5 implements KeySelector<CustomType, Integer> {
+		@Override
+		public Integer getKey(CustomType value) {
+			return value.myInt;
+		}
+	}
+
+	private static class KeySelector6 implements KeySelector<CustomType, Integer> {
+		@Override
+		public Integer getKey(CustomType value) {
+			return value.myInt;
+		}
+	}
+
+	@Test
+	public void testUDFJoinOnTuplesWithTupleReturningKeySelectors() throws Exception {
+		/*
+		 * UDF Join on tuples with tuple-returning key selectors
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+		DataSet<Tuple2<String, String>> joinDs =
+				ds1.join(ds2)
+						.where(new KeySelector3())
+						.equalTo(new KeySelector4())
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt wie gehts?\n" +
+				"Hello world,ABC\n" +
+				"I am fine.,HIJ\n" +
+				"I am fine.,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class KeySelector3 implements KeySelector<Tuple3<Integer, Long, String>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple3<Integer, Long, String> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f1);
+		}
+	}
+
+	private static class KeySelector4 implements KeySelector<Tuple5<Integer, Long, Integer, String, Long>, Tuple2<Integer, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Long> getKey(Tuple5<Integer, Long, Integer, String, Long> t) {
+			return new Tuple2<Integer, Long>(t.f0, t.f4);
+		}
+	}
+
+	@Test
+	public void testJoinNestedPojoAgainstTupleSelectedUsingString() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected using a string)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.join(ds2).where("nestedPojo.longNumber").equalTo("f6");
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinNestedPojoAgainstTupleSelectedUsingInteger() throws Exception {
+		/*
+		 * Join nested pojo against tuple (selected as an integer)
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.join(ds2).where("nestedPojo.longNumber").equalTo(6); // <--- difference!
+
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testSelectingMultipleFieldsUsingExpressionLanguage() throws Exception {
+		/*
+		 * selecting multiple fields using expression language
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.join(ds2).where("nestedPojo.longNumber", "number", "str").equalTo("f6", "f0", "f1");
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTuple() throws Exception {
+		/*
+		 * nested into tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.join(ds2).where("nestedPojo.longNumber", "number", "nestedTupleWithCustom.f0").equalTo("f6", "f0", "f2");
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNestedIntoTupleIntoPojo() throws Exception {
+		/*
+		 * nested into tuple into pojo
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Integer, String, Integer, Integer, Long, String, Long>> ds2 = CollectionDataSets.getSmallTuplebasedDataSet(env);
+		DataSet<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> joinDs =
+				ds1.join(ds2).where("nestedTupleWithCustom.f0", "nestedTupleWithCustom.f1.myInt", "nestedTupleWithCustom.f1.myLong").equalTo("f2", "f3", "f4");
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Integer, String, Integer, Integer, Long, String, Long>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(1,First,10,100,1000,One,10000)\n" +
+				"2 Second (20,200,2000,Two) 20000,(2,Second,20,200,2000,Two,20000)\n" +
+				"3 Third (30,300,3000,Three) 30000,(3,Third,30,300,3000,Three,30000)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNonPojoToVerifyFullTupleKeys() throws Exception {
+		/*
+		 * Non-POJO test to verify that full-tuple keys are working.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.join(ds2).where(0).equalTo("f0.f0", "f0.f1"); // key is now Tuple2<Integer, Integer>
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+
+	}
+
+	@Test
+	public void testNonPojoToVerifyNestedTupleElementSelection() throws Exception {
+		/*
+		 * Non-POJO test to verify "nested" tuple-element selection.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds1 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Integer, Integer>, String>> ds2 = CollectionDataSets.getSmallNestedTupleDataSet(env);
+		DataSet<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> joinDs =
+				ds1.join(ds2).where("f0.f0").equalTo("f0.f0"); // key is now Integer from Tuple2<Integer, Integer>
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple2<Integer, Integer>, String>, Tuple2<Tuple2<Integer, Integer>, String>>> result = joinDs.collect();
+
+		String expected = "((1,1),one),((1,1),one)\n" +
+				"((2,2),two),((2,2),two)\n" +
+				"((3,3),three),((3,3),three)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testFullPojoWithFullTuple() throws Exception {
+		/*
+		 * full pojo with full tuple
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<POJO> ds1 = CollectionDataSets.getSmallPojoDataSet(env);
+		DataSet<Tuple7<Long, Integer, Integer, Long, String, Integer, String>> ds2 = CollectionDataSets.getSmallTuplebasedDataSetMatchingPojo(env);
+		DataSet<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> joinDs =
+				ds1.join(ds2).where("*").equalTo("*");
+
+		env.setParallelism(1);
+		List<Tuple2<POJO, Tuple7<Long, Integer, Integer, Long, String, Integer, String>>> result = joinDs.collect();
+
+		String expected = "1 First (10,100,1000,One) 10000,(10000,10,100,1000,One,1,First)\n" +
+				"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n" +
+				"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testNonPojoToVerifyNestedTupleElementSelectionWithFirstKeyFieldGreaterThanZero()
+			throws Exception {
+		/*
+		 * Non-POJO test to verify "nested" tuple-element selection with the first key field greater than 0.
+		 */
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>> ds2 = ds1.join(ds1).where(0).equalTo(0);
+		DataSet<Tuple2<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>, Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>>> joinDs =
+				ds2.join(ds2).where("f1.f0").equalTo("f0.f0");
+
+		env.setParallelism(1);
+		List<Tuple2<Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>, Tuple2<Tuple3<Integer, Long, String>, Tuple3<Integer, Long, String>>>> result = joinDs.collect();
+
+		String expected = "((1,1,Hi),(1,1,Hi)),((1,1,Hi),(1,1,Hi))\n" +
+				"((2,2,Hello),(2,2,Hello)),((2,2,Hello),(2,2,Hello))\n" +
+				"((3,2,Hello world),(3,2,Hello world)),((3,2,Hello world),(3,2,Hello world))\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType1() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Integer> ds2 = env.fromElements(1, 2);
+
+		DataSet<Tuple2<Tuple3<Integer, Long, String>, Integer>> joinDs = ds1.join(ds2).where(0).equalTo("*");
+
+		List<Tuple2<Tuple3<Integer, Long, String>, Integer>> result = joinDs.collect();
+
+		String expected = "(1,1,Hi),1\n" +
+				"(2,2,Hello),2";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithAtomicType2() throws Exception {
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Integer> ds1 = env.fromElements(1, 2);
+		DataSet<Tuple3<Integer, Long, String>> ds2 = CollectionDataSets.getSmall3TupleDataSet(env);
+
+		DataSet<Tuple2<Integer, Tuple3<Integer, Long, String>>> joinDs = ds1.join(ds2).where("*").equalTo(0);
+
+		List<Tuple2<Integer, Tuple3<Integer, Long, String>>> result = joinDs.collect();
+
+		String expected = "1,(1,1,Hi)\n" +
+				"2,(2,2,Hello)";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	@Test
+	public void testJoinWithRangePartitioning() throws Exception {
+		/*
+		 * Test Join on tuples with multiple key field positions and same customized distribution
+		 */
+
+		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		env.setParallelism(4);
+		TestDistribution testDis = new TestDistribution();
+		DataSet<Tuple2<String, String>> joinDs =
+				DataSetUtils.partitionByRange(ds1, testDis, 0, 1)
+						.join(DataSetUtils.partitionByRange(ds2, testDis, 0, 4))
+						.where(0, 1)
+						.equalTo(0, 4)
+						.with(new T3T5FlatJoin());
+
+		List<Tuple2<String, String>> result = joinDs.collect();
+
+		String expected = "Hi,Hallo\n" +
+				"Hello,Hallo Welt\n" +
+				"Hello world,Hallo Welt wie gehts?\n" +
+				"Hello world,ABC\n" +
+				"I am fine.,HIJ\n" +
+				"I am fine.,IJK\n";
+
+		compareResultAsTuples(result, expected);
+	}
+
+	private static class T3T5FlatJoin implements FlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple2<String, String>> {
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second,
+				Collector<Tuple2<String, String>> out) {
+
+			out.collect(new Tuple2<String, String>(first.f2, second.f3));
+		}
+
+	}
+
+	private static class LeftReturningJoin implements JoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<Integer, Long, String>> {
+
+		@Override
+		public Tuple3<Integer, Long, String> join(Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second) {
+
+			return first;
+		}
+	}
+
+	private static class RightReturningJoin implements JoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple5<Integer, Long, Integer, String, Long>> {
+
+		@Override
+		public Tuple5<Integer, Long, Integer, String, Long> join(Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second) {
+
+			return second;
+		}
+	}
+
+	private static class T3T5BCJoin extends RichFlatJoinFunction<Tuple3<Integer, Long, String>, Tuple5<Integer, Long, Integer, String, Long>, Tuple3<String, String, Integer>> {
+
+		private int broadcast;
+
+		@Override
+		public void open(Configuration config) {
+
+			Collection<Integer> ints = this.getRuntimeContext().getBroadcastVariable("ints");
+			int sum = 0;
+			for (Integer i : ints) {
+				sum += i;
+			}
+			broadcast = sum;
+
+		}
+
+		/*
+		@Override
+		public Tuple3<String, String, Integer> join(
+				Tuple3<Integer, Long, String> first,
+				Tuple5<Integer, Long, Integer, String, Long> second) {
+
+			return new Tuple3<String, String, Integer>(first.f2, second.f3, broadcast);
+		}
+		 */
+
+		@Override
+		public void join(Tuple3<Integer, Long, String> first, Tuple5<Integer, Long, Integer, String, Long> second, Collector<Tuple3<String, String, Integer>> out) throws Exception {
+			out.collect(new Tuple3<String, String, Integer>(first.f2, second.f3, broadcast));
+		}
+	}
+
+	private static class T3CustJoin implements JoinFunction<Tuple3<Integer, Long, String>, CustomType, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(Tuple3<Integer, Long, String> first,
+				CustomType second) {
+
+			return new Tuple2<String, String>(first.f2, second.myString);
+		}
+	}
+
+	private static class CustT3Join implements JoinFunction<CustomType, Tuple3<Integer, Long, String>, Tuple2<String, String>> {
+
+		@Override
+		public Tuple2<String, String> join(CustomType first, Tuple3<Integer, Long, String> second) {
+
+			return new Tuple2<String, String>(first.myString, second.f2);
+		}
+	}
+
+	/**
+	 * Test data distribution.
+	 */
+	public static class TestDistribution implements DataDistribution {
+		public Object[][] boundaries = new Object[][]{
+				new Object[]{2, 2L},
+				new Object[]{5, 4L},
+				new Object[]{10, 12L},
+				new Object[]{21, 6L}
+		};
+
+		public TestDistribution() {
+		}
+
+		@Override
+		public Object[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
+			return boundaries[bucketNum];
+		}
+
+		@Override
+		public int getNumberOfFields() {
+			return 2;
+		}
+
+		@Override
+		public TypeInformation[] getKeyTypes() {
+			return new TypeInformation[]{BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO};
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj instanceof TestDistribution;
+		}
+	}
+}


[02/22] flink git commit: [FLINK-6731] [tests] Activate strict checkstyle for flink-tests

Posted by gr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
index 765eae5..f73bf42 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
@@ -20,31 +20,33 @@ package org.apache.flink.test.streaming.runtime;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.streaming.api.functions.ProcessFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.OutputTag;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.test.streaming.runtime.util.TestListResultSink;
 import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
 import javax.annotation.Nullable;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -400,7 +402,6 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
 		assertEquals(Arrays.asList(1, 2, 3, 4, 5), resultSink.getSortedResult());
 	}
 
-
 	/**
 	 * Test ProcessFunction side outputs with wrong {@code OutputTag}.
 	 */
@@ -450,7 +451,7 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
 
 	private static class TestKeySelector implements KeySelector<Integer, Integer> {
 		private static final long serialVersionUID = 1L;
-		
+
 		@Override
 		public Integer getKey(Integer value) throws Exception {
 			return value;
@@ -458,7 +459,7 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
 	}
 
 	/**
-	 * Test window late arriving events stream
+	 * Test window late arriving events stream.
 	 */
 	@Test
 	public void testAllWindowLateArrivingEvents() throws Exception {
@@ -478,10 +479,10 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
 				.sideOutputLateData(lateDataTag)
 				.apply(new AllWindowFunction<Integer, Integer, TimeWindow>() {
 					private static final long serialVersionUID = 1L;
-					
+
 					@Override
 					public void apply(TimeWindow window, Iterable<Integer> values, Collector<Integer> out) throws Exception {
-							for(Integer val : values) {
+							for (Integer val : values) {
 								out.collect(val);
 							}
 					}
@@ -528,7 +529,7 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
 
 					@Override
 					public void apply(Integer key, TimeWindow window, Iterable<Integer> input, Collector<String> out) throws Exception {
-						for(Integer val : input) {
+						for (Integer val : input) {
 							out.collect(String.valueOf(key) + "-" + String.valueOf(val));
 						}
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
index ce342c0..f5c769d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+
 import org.junit.Test;
 
 import java.io.IOException;
@@ -42,6 +43,9 @@ import java.io.IOException;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Integration tests for {@link OperatorStateBackend}.
+ */
 public class StateBackendITCase extends StreamingMultipleProgramsTestBase {
 
 	/**
@@ -58,10 +62,9 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase {
 		see.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		see.setStateBackend(new FailingStateBackend());
 
-
 		see.fromElements(new Tuple2<>("Hello", 1))
 			.keyBy(0)
-			.map(new RichMapFunction<Tuple2<String,Integer>, String>() {
+			.map(new RichMapFunction<Tuple2<String, Integer>, String>() {
 				private static final long serialVersionUID = 1L;
 
 				@Override
@@ -87,7 +90,7 @@ public class StateBackendITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	public static class FailingStateBackend extends AbstractStateBackend {
+	private static class FailingStateBackend extends AbstractStateBackend {
 		private static final long serialVersionUID = 1L;
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskTimerITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskTimerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskTimerITCase.java
index 48e6fae..cb31f2b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskTimerITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskTimerITCase.java
@@ -29,8 +29,8 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
 import org.apache.flink.streaming.runtime.tasks.TimerException;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 
@@ -46,27 +46,25 @@ import java.util.concurrent.Semaphore;
 /**
  * Tests for the timer service of {@code StreamTask}.
  *
- * <p>
- * These tests ensure that exceptions are properly forwarded from the timer thread to
+ * <p>These tests ensure that exceptions are properly forwarded from the timer thread to
  * the task thread and that operator methods are not invoked concurrently.
  */
 @RunWith(Parameterized.class)
 public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 
 	private final TimeCharacteristic timeCharacteristic;
-	
+
 	public StreamTaskTimerITCase(TimeCharacteristic characteristic) {
 		timeCharacteristic = characteristic;
 	}
 
-
 	/**
 	 * Note: this test fails if we don't check for exceptions in the source contexts and do not
 	 * synchronize in the source contexts.
 	 */
 	@Test
 	public void testOperatorChainedToSource() throws Exception {
-		
+
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setStreamTimeCharacteristic(timeCharacteristic);
 		env.setParallelism(1);
@@ -134,7 +132,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 		}
 		Assert.assertTrue(testSuccess);
 	}
-	
+
 	@Test
 	public void testTwoInputOperatorWithoutChaining() throws Exception {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
@@ -171,7 +169,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 		Assert.assertTrue(testSuccess);
 	}
 
-	public static class TimerOperator extends AbstractStreamOperator<String> implements OneInputStreamOperator<String, String>, ProcessingTimeCallback {
+	private static class TimerOperator extends AbstractStreamOperator<String> implements OneInputStreamOperator<String, String>, ProcessingTimeCallback {
 		private static final long serialVersionUID = 1L;
 
 		int numTimers = 0;
@@ -196,7 +194,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 				first = false;
 			}
 			numElements++;
-			
+
 			semaphore.release();
 		}
 
@@ -230,7 +228,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	public static class TwoInputTimerOperator extends AbstractStreamOperator<String> implements TwoInputStreamOperator<String, String, String>, ProcessingTimeCallback {
+	private static class TwoInputTimerOperator extends AbstractStreamOperator<String> implements TwoInputStreamOperator<String, String, String>, ProcessingTimeCallback {
 		private static final long serialVersionUID = 1L;
 
 		int numTimers = 0;
@@ -274,7 +272,6 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 			semaphore.release();
 		}
 
-
 		@Override
 		public void onProcessingTime(long time) throws Exception {
 			if (!semaphore.tryAcquire()) {
@@ -307,7 +304,6 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-
 	private static class InfiniteTestSource implements SourceFunction<String> {
 		private static final long serialVersionUID = 1L;
 		private volatile boolean running = true;
@@ -324,7 +320,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
 			running = false;
 		}
 	}
-	
+
 	// ------------------------------------------------------------------------
 	//  parametrization
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
index 45686ef..9b1dd2a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
@@ -32,12 +32,12 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -75,7 +75,6 @@ public class TimestampITCase extends TestLogger {
 	// this is used in some tests to synchronize
 	static MultiShotLatch latch;
 
-
 	private static LocalFlinkMiniCluster cluster;
 
 	@Before
@@ -84,7 +83,6 @@ public class TimestampITCase extends TestLogger {
 		latch = new MultiShotLatch();
 	}
 
-
 	@BeforeClass
 	public static void startCluster() {
 		Configuration config = new Configuration();
@@ -111,28 +109,26 @@ public class TimestampITCase extends TestLogger {
 	 * These check whether custom timestamp emission works at sources and also whether timestamps
 	 * arrive at operators throughout a topology.
 	 *
-	 * <p>
-	 * This also checks whether watermarks keep propagating if a source closes early.
+	 * <p>This also checks whether watermarks keep propagating if a source closes early.
 	 *
-	 * <p>
-	 * This only uses map to test the workings of watermarks in a complete, running topology. All
+	 * <p>This only uses map to test the workings of watermarks in a complete, running topology. All
 	 * tasks and stream operators have dedicated tests that test the watermark propagation
 	 * behaviour.
 	 */
 	@Test
 	public void testWatermarkPropagation() throws Exception {
-		final int NUM_WATERMARKS = 10;
+		final int numWatermarks = 10;
 
 		long initialTime = 0L;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		
+
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
-		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS));
-		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS / 2));
+		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(initialTime, numWatermarks));
+		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(initialTime, numWatermarks / 2));
 
 		source1.union(source2)
 				.map(new IdentityMap())
@@ -146,31 +142,31 @@ public class TimestampITCase extends TestLogger {
 		for (int i = 0; i < PARALLELISM; i++) {
 			// we are only guaranteed to see NUM_WATERMARKS / 2 watermarks because the
 			// other source stops emitting after that
-			for (int j = 0; j < NUM_WATERMARKS / 2; j++) {
+			for (int j = 0; j < numWatermarks / 2; j++) {
 				if (!CustomOperator.finalWatermarks[i].get(j).equals(new Watermark(initialTime + j))) {
 					System.err.println("All Watermarks: ");
-					for (int k = 0; k <= NUM_WATERMARKS / 2; k++) {
+					for (int k = 0; k <= numWatermarks / 2; k++) {
 						System.err.println(CustomOperator.finalWatermarks[i].get(k));
 					}
 
 					fail("Wrong watermark.");
 				}
 			}
-			
+
 			assertEquals(Watermark.MAX_WATERMARK,
-					CustomOperator.finalWatermarks[i].get(CustomOperator.finalWatermarks[i].size()-1));
+					CustomOperator.finalWatermarks[i].get(CustomOperator.finalWatermarks[i].size() - 1));
 		}
 	}
 
 	@Test
 	public void testWatermarkPropagationNoFinalWatermarkOnStop() throws Exception {
-		
+
 		// for this test to work, we need to be sure that no other jobs are being executed
 		while (!cluster.getCurrentlyRunningJobsJava().isEmpty()) {
 			Thread.sleep(100);
 		}
-		
-		final int NUM_WATERMARKS = 10;
+
+		final int numWatermarks = 10;
 
 		long initialTime = 0L;
 
@@ -180,8 +176,8 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
-		DataStream<Integer> source1 = env.addSource(new MyTimestampSourceInfinite(initialTime, NUM_WATERMARKS));
-		DataStream<Integer> source2 = env.addSource(new MyTimestampSourceInfinite(initialTime, NUM_WATERMARKS / 2));
+		DataStream<Integer> source1 = env.addSource(new MyTimestampSourceInfinite(initialTime, numWatermarks));
+		DataStream<Integer> source2 = env.addSource(new MyTimestampSourceInfinite(initialTime, numWatermarks / 2));
 
 		source1.union(source2)
 				.map(new IdentityMap())
@@ -200,7 +196,7 @@ public class TimestampITCase extends TestLogger {
 					}
 
 					JobID id = running.get(0);
-					
+
 					// send stop until the job is stopped
 					do {
 						try {
@@ -210,7 +206,7 @@ public class TimestampITCase extends TestLogger {
 							if (e.getCause() instanceof IllegalStateException) {
 								// this means the job is not yet ready to be stopped,
 								// for example because it is still in CREATED state
-								// we ignore the exception 
+								// we ignore the exception
 							} else {
 								// other problem
 								throw e;
@@ -225,29 +221,29 @@ public class TimestampITCase extends TestLogger {
 				}
 			}
 		}.start();
-		
+
 		env.execute();
 
 		// verify that all the watermarks arrived at the final custom operator
 		for (List<Watermark> subtaskWatermarks : CustomOperator.finalWatermarks) {
-			
+
 			// we are only guaranteed to see NUM_WATERMARKS / 2 watermarks because the
 			// other source stops emitting after that
 			for (int j = 0; j < subtaskWatermarks.size(); j++) {
 				if (subtaskWatermarks.get(j).getTimestamp() != initialTime + j) {
 					System.err.println("All Watermarks: ");
-					for (int k = 0; k <= NUM_WATERMARKS / 2; k++) {
+					for (int k = 0; k <= numWatermarks / 2; k++) {
 						System.err.println(subtaskWatermarks.get(k));
 					}
 
 					fail("Wrong watermark.");
 				}
 			}
-			
+
 			// if there are watermarks, the final one must not be the MAX watermark
 			if (subtaskWatermarks.size() > 0) {
 				assertNotEquals(Watermark.MAX_WATERMARK,
-						subtaskWatermarks.get(subtaskWatermarks.size()-1));
+						subtaskWatermarks.get(subtaskWatermarks.size() - 1));
 			}
 		}
 	}
@@ -258,8 +254,7 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testTimestampHandling() throws Exception {
-		final int NUM_ELEMENTS = 10;
-
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -267,8 +262,8 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
 
-		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS));
-		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS));
+		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(0L, numElements));
+		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(0L, numElements));
 
 		source1
 				.map(new IdentityMap())
@@ -276,7 +271,6 @@ public class TimestampITCase extends TestLogger {
 				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator())
 				.addSink(new DiscardingSink<Integer>());
 
-
 		env.execute();
 	}
 
@@ -285,23 +279,23 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testDisabledTimestamps() throws Exception {
-		final int NUM_ELEMENTS = 10;
-		
+		final int numElements = 10;
+
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		
+
 		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 		env.setParallelism(PARALLELISM);
 		env.getConfig().disableSysoutLogging();
-		
-		DataStream<Integer> source1 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS));
-		DataStream<Integer> source2 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS));
+
+		DataStream<Integer> source1 = env.addSource(new MyNonWatermarkingSource(numElements));
+		DataStream<Integer> source2 = env.addSource(new MyNonWatermarkingSource(numElements));
 
 		source1
 				.map(new IdentityMap())
 				.connect(source2).map(new IdentityCoMap())
 				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator())
 				.addSink(new DiscardingSink<Integer>());
-		
+
 		env.execute();
 	}
 
@@ -312,7 +306,7 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testTimestampExtractorWithAutoInterval() throws Exception {
-		final int NUM_ELEMENTS = 10;
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -321,12 +315,11 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(1);
 		env.getConfig().disableSysoutLogging();
 
-
 		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
 			@Override
 			public void run(SourceContext<Integer> ctx) throws Exception {
 				int index = 1;
-				while (index <= NUM_ELEMENTS) {
+				while (index <= numElements) {
 					ctx.collect(index);
 					latch.await();
 					index++;
@@ -357,15 +350,15 @@ public class TimestampITCase extends TestLogger {
 		env.execute();
 
 		// verify that we get NUM_ELEMENTS watermarks
-		for (int j = 0; j < NUM_ELEMENTS; j++) {
+		for (int j = 0; j < numElements; j++) {
 			if (!CustomOperator.finalWatermarks[0].get(j).equals(new Watermark(j))) {
 				long wm = CustomOperator.finalWatermarks[0].get(j).getTimestamp();
 				Assert.fail("Wrong watermark. Expected: " + j + " Found: " + wm + " All: " + CustomOperator.finalWatermarks[0]);
 			}
 		}
-		
+
 		// the input is finite, so it should have a MAX Watermark
-		assertEquals(Watermark.MAX_WATERMARK, 
+		assertEquals(Watermark.MAX_WATERMARK,
 				CustomOperator.finalWatermarks[0].get(CustomOperator.finalWatermarks[0].size() - 1));
 	}
 
@@ -376,7 +369,7 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testTimestampExtractorWithCustomWatermarkEmit() throws Exception {
-		final int NUM_ELEMENTS = 10;
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -385,12 +378,11 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(1);
 		env.getConfig().disableSysoutLogging();
 
-
 		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
 			@Override
 			public void run(SourceContext<Integer> ctx) throws Exception {
 				int index = 1;
-				while (index <= NUM_ELEMENTS) {
+				while (index <= numElements) {
 					ctx.collect(index);
 					latch.await();
 					index++;
@@ -403,7 +395,7 @@ public class TimestampITCase extends TestLogger {
 
 		source1
 				.assignTimestampsAndWatermarks(new AssignerWithPunctuatedWatermarks<Integer>() {
-					
+
 					@Override
 					public long extractTimestamp(Integer element, long currentTimestamp) {
 						return element;
@@ -417,11 +409,10 @@ public class TimestampITCase extends TestLogger {
 				.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(true))
 				.transform("Timestamp Check", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator());
 
-
 		env.execute();
 
 		// verify that we get NUM_ELEMENTS watermarks
-		for (int j = 0; j < NUM_ELEMENTS; j++) {
+		for (int j = 0; j < numElements; j++) {
 			if (!CustomOperator.finalWatermarks[0].get(j).equals(new Watermark(j))) {
 				Assert.fail("Wrong watermark.");
 			}
@@ -433,12 +424,11 @@ public class TimestampITCase extends TestLogger {
 	}
 
 	/**
-	 * This test verifies that the timestamp extractor does not emit decreasing watermarks even
-	 *
+	 * This test verifies that the timestamp extractor does not emit decreasing watermarks.
 	 */
 	@Test
 	public void testTimestampExtractorWithDecreasingCustomWatermarkEmit() throws Exception {
-		final int NUM_ELEMENTS = 10;
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -447,12 +437,11 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(1);
 		env.getConfig().disableSysoutLogging();
 
-
 		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
 			@Override
 			public void run(SourceContext<Integer> ctx) throws Exception {
 				int index = 1;
-				while (index <= NUM_ELEMENTS) {
+				while (index <= numElements) {
 					ctx.collect(index);
 					Thread.sleep(100);
 					ctx.collect(index - 1);
@@ -481,11 +470,10 @@ public class TimestampITCase extends TestLogger {
 				.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(true))
 				.transform("Timestamp Check", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator());
 
-
 		env.execute();
 
 		// verify that we get NUM_ELEMENTS watermarks
-		for (int j = 0; j < NUM_ELEMENTS; j++) {
+		for (int j = 0; j < numElements; j++) {
 			if (!CustomOperator.finalWatermarks[0].get(j).equals(new Watermark(j))) {
 				Assert.fail("Wrong watermark.");
 			}
@@ -500,7 +488,7 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testTimestampExtractorWithLongMaxWatermarkFromSource() throws Exception {
-		final int NUM_ELEMENTS = 10;
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
@@ -509,16 +497,15 @@ public class TimestampITCase extends TestLogger {
 		env.setParallelism(2);
 		env.getConfig().disableSysoutLogging();
 
-
 		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
 			@Override
 			public void run(SourceContext<Integer> ctx) throws Exception {
 				int index = 1;
-				while (index <= NUM_ELEMENTS) {
+				while (index <= numElements) {
 					ctx.collectWithTimestamp(index, index);
 					ctx.collectWithTimestamp(index - 1, index - 1);
 					index++;
-					ctx.emitWatermark(new Watermark(index-2));
+					ctx.emitWatermark(new Watermark(index - 2));
 				}
 
 				// emit the final Long.MAX_VALUE watermark, do it twice and verify that
@@ -546,7 +533,6 @@ public class TimestampITCase extends TestLogger {
 				})
 			.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(true));
 
-
 		env.execute();
 
 		Assert.assertTrue(CustomOperator.finalWatermarks[0].size() == 1);
@@ -555,12 +541,12 @@ public class TimestampITCase extends TestLogger {
 
 	/**
 	 * This test verifies that the timestamp extractor forwards Long.MAX_VALUE watermarks.
-	 * 
-	 * Same test as before, but using a different timestamp extractor
+	 *
+	 * <p>Same test as before, but using a different timestamp extractor.
 	 */
 	@Test
 	public void testTimestampExtractorWithLongMaxWatermarkFromSource2() throws Exception {
-		final int NUM_ELEMENTS = 10;
+		final int numElements = 10;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment
 				.getExecutionEnvironment();
@@ -574,11 +560,11 @@ public class TimestampITCase extends TestLogger {
 			@Override
 			public void run(SourceContext<Integer> ctx) throws Exception {
 				int index = 1;
-				while (index <= NUM_ELEMENTS) {
+				while (index <= numElements) {
 					ctx.collectWithTimestamp(index, index);
 					ctx.collectWithTimestamp(index - 1, index - 1);
 					index++;
-					ctx.emitWatermark(new Watermark(index-2));
+					ctx.emitWatermark(new Watermark(index - 2));
 				}
 
 				// emit the final Long.MAX_VALUE watermark, do it twice and verify that
@@ -605,7 +591,7 @@ public class TimestampITCase extends TestLogger {
 					}
 				})
 				.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(true));
-		
+
 		env.execute();
 
 		Assert.assertTrue(CustomOperator.finalWatermarks[0].size() == 1);
@@ -618,9 +604,9 @@ public class TimestampITCase extends TestLogger {
 	 */
 	@Test
 	public void testEventTimeSourceWithProcessingTime() throws Exception {
-		StreamExecutionEnvironment env = 
+		StreamExecutionEnvironment env =
 				StreamExecutionEnvironment.getExecutionEnvironment();
-		
+
 		env.setParallelism(2);
 		env.getConfig().disableSysoutLogging();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
@@ -637,7 +623,7 @@ public class TimestampITCase extends TestLogger {
 		// other tests, so it normally emits watermarks
 		Assert.assertTrue(CustomOperator.finalWatermarks[0].size() == 0);
 	}
-	
+
 	@Test
 	public void testErrorOnEventTimeOverProcessingTime() {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
@@ -646,7 +632,7 @@ public class TimestampITCase extends TestLogger {
 		env.getConfig().disableSysoutLogging();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 
-		DataStream<Tuple2<String, Integer>> source1 = 
+		DataStream<Tuple2<String, Integer>> source1 =
 				env.fromElements(new Tuple2<>("a", 1), new Tuple2<>("b", 2));
 
 		source1
@@ -701,9 +687,9 @@ public class TimestampITCase extends TestLogger {
 	// ------------------------------------------------------------------------
 	//  Custom Operators and Functions
 	// ------------------------------------------------------------------------
-	
+
 	@SuppressWarnings("unchecked")
-	public static class CustomOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
+	private static class CustomOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
 
 		List<Watermark> watermarks;
 		public static List<Watermark>[] finalWatermarks = new List[PARALLELISM];
@@ -749,7 +735,7 @@ public class TimestampITCase extends TestLogger {
 		}
 	}
 
-	public static class TimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
+	private static class TimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
 
 		public TimestampCheckingOperator() {
 			setChainingStrategy(ChainingStrategy.ALWAYS);
@@ -764,7 +750,7 @@ public class TimestampITCase extends TestLogger {
 		}
 	}
 
-	public static class DisabledTimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
+	private static class DisabledTimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
 
 		@Override
 		public void processElement(StreamRecord<Integer> element) throws Exception {
@@ -775,7 +761,7 @@ public class TimestampITCase extends TestLogger {
 		}
 	}
 
-	public static class IdentityCoMap implements CoMapFunction<Integer, Integer, Integer> {
+	private static class IdentityCoMap implements CoMapFunction<Integer, Integer, Integer> {
 		@Override
 		public Integer map1(Integer value) throws Exception {
 			return value;
@@ -787,14 +773,14 @@ public class TimestampITCase extends TestLogger {
 		}
 	}
 
-	public static class IdentityMap implements MapFunction<Integer, Integer> {
+	private static class IdentityMap implements MapFunction<Integer, Integer> {
 		@Override
 		public Integer map(Integer value) throws Exception {
 			return value;
 		}
 	}
 
-	public static class MyTimestampSource implements SourceFunction<Integer> {
+	private static class MyTimestampSource implements SourceFunction<Integer> {
 
 		private final long initialTime;
 		private final int numWatermarks;
@@ -816,13 +802,13 @@ public class TimestampITCase extends TestLogger {
 		public void cancel() {}
 	}
 
-	public static class MyTimestampSourceInfinite implements SourceFunction<Integer>, StoppableFunction {
+	private static class MyTimestampSourceInfinite implements SourceFunction<Integer>, StoppableFunction {
 
 		private final long initialTime;
 		private final int numWatermarks;
 
 		private volatile boolean running = true;
-		
+
 		public MyTimestampSourceInfinite(long initialTime, int numWatermarks) {
 			this.initialTime = initialTime;
 			this.numWatermarks = numWatermarks;
@@ -834,7 +820,7 @@ public class TimestampITCase extends TestLogger {
 				ctx.collectWithTimestamp(i, initialTime + i);
 				ctx.emitWatermark(new Watermark(initialTime + i));
 			}
-			
+
 			while (running) {
 				Thread.sleep(20);
 			}
@@ -851,7 +837,7 @@ public class TimestampITCase extends TestLogger {
 		}
 	}
 
-	public static class MyNonWatermarkingSource implements SourceFunction<Integer> {
+	private static class MyNonWatermarkingSource implements SourceFunction<Integer> {
 
 		int numWatermarks;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/WindowFoldITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/WindowFoldITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/WindowFoldITCase.java
index 903179d..b347c16 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/WindowFoldITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/WindowFoldITCase.java
@@ -35,6 +35,7 @@ import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -84,7 +85,7 @@ public class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
 
 			@Override
 			public void cancel() {}
-			
+
 		}).assignTimestampsAndWatermarks(new Tuple2TimestampExtractor());
 
 		source1

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/EvenOddOutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/EvenOddOutputSelector.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/EvenOddOutputSelector.java
index 8fc8372..804c19b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/EvenOddOutputSelector.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/EvenOddOutputSelector.java
@@ -15,12 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.streaming.runtime.util;
 
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 
 import java.util.Collections;
 
+/**
+ * {@link OutputSelector} mapping integers to "even" and "odd" streams.
+ */
 public class EvenOddOutputSelector implements OutputSelector<Integer> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/NoOpIntMap.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/NoOpIntMap.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/NoOpIntMap.java
index 6667446..2699ba8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/NoOpIntMap.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/NoOpIntMap.java
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.streaming.runtime.util;
 
 import org.apache.flink.api.common.functions.MapFunction;
 
+/**
+ * Identity mapper.
+ */
 public class NoOpIntMap implements MapFunction<Integer, Integer> {
 	private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java
index 21d5294..8e60735 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.streaming.runtime.util;
 
 import org.apache.flink.configuration.Configuration;
@@ -25,6 +26,11 @@ import java.util.List;
 
 import static org.junit.Assert.assertTrue;
 
+/**
+ * SinkFunction asserting that at least one record was collected.
+ *
+ * @param <T> element type
+ */
 public final class ReceiveCheckNoOpSink<T> extends RichSinkFunction<T> {
 	private List<T> received;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java
index 3fabb4b..8d81ede 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java
@@ -17,15 +17,18 @@
 
 package org.apache.flink.test.streaming.runtime.util;
 
-import java.util.Collections;
-import java.util.Comparator;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.TreeSet;
 
+/**
+ * Thread-safe sink for collecting elements into an on-heap list.
+ *
+ * @param <T> element type
+ */
 public class TestListResultSink<T> extends RichSinkFunction<T> {
 
 	private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListWrapper.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListWrapper.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListWrapper.java
index 19ca8eb..b0b03e8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListWrapper.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/TestListWrapper.java
@@ -21,6 +21,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+/**
+ * Catalog for lists stored by {@link TestListResultSink}.
+ */
 public class TestListWrapper {
 
 	private static TestListWrapper instance;
@@ -61,4 +64,4 @@ public class TestListWrapper {
 		return list;
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/testfunctions/Tokenizer.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/testfunctions/Tokenizer.java b/flink-tests/src/test/java/org/apache/flink/test/testfunctions/Tokenizer.java
index 9b3764d..e0f8b15 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/testfunctions/Tokenizer.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/testfunctions/Tokenizer.java
@@ -22,6 +22,9 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 
+/**
+ * FlatMap for splitting strings.
+ */
 public final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
index e4dd535..54fe879 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java
@@ -50,6 +50,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.util.DynamicCodeLoadingException;
 import org.apache.flink.util.StateMigrationException;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -58,6 +59,7 @@ import org.junit.runners.Parameterized;
 
 import javax.tools.JavaCompiler;
 import javax.tools.ToolProvider;
+
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
@@ -73,6 +75,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Tests the state migration behaviour when the underlying POJO type changes
+ * and one tries to recover from old state.
+ */
 @RunWith(Parameterized.class)
 public class PojoSerializerUpgradeTest extends TestLogger {
 
@@ -168,7 +174,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 		"@Override public String toString() {return \"(\" + a + \")\";}}";
 
 	/**
-	 * We should be able to handle a changed field order of a POJO as keyed state
+	 * We should be able to handle a changed field order of a POJO as keyed state.
 	 */
 	@Test
 	public void testChangedFieldOrderWithKeyedState() throws Exception {
@@ -176,7 +182,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * We should be able to handle a changed field order of a POJO as operator state
+	 * We should be able to handle a changed field order of a POJO as operator state.
 	 */
 	@Test
 	public void testChangedFieldOrderWithOperatorState() throws Exception {
@@ -184,7 +190,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Changing field types of a POJO as keyed state should require a state migration
+	 * Changing field types of a POJO as keyed state should require a state migration.
 	 */
 	@Test
 	public void testChangedFieldTypesWithKeyedState() throws Exception {
@@ -201,7 +207,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Changing field types of a POJO as operator state should require a state migration
+	 * Changing field types of a POJO as operator state should require a state migration.
 	 */
 	@Test
 	public void testChangedFieldTypesWithOperatorState() throws Exception {
@@ -218,7 +224,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Adding fields to a POJO as keyed state should require a state migration
+	 * Adding fields to a POJO as keyed state should require a state migration.
 	 */
 	@Test
 	public void testAdditionalFieldWithKeyedState() throws Exception {
@@ -235,7 +241,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Adding fields to a POJO as operator state should require a state migration
+	 * Adding fields to a POJO as operator state should require a state migration.
 	 */
 	@Test
 	public void testAdditionalFieldWithOperatorState() throws Exception {
@@ -252,7 +258,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Removing fields from a POJO as keyed state should require a state migration
+	 * Removing fields from a POJO as keyed state should require a state migration.
 	 */
 	@Test
 	public void testMissingFieldWithKeyedState() throws Exception {
@@ -269,7 +275,7 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 	}
 
 	/**
-	 * Removing fields from a POJO as operator state should require a state migration
+	 * Removing fields from a POJO as operator state should require a state migration.
 	 */
 	@Test
 	public void testMissingFieldWithOperatorState() throws Exception {
@@ -380,7 +386,6 @@ public class PojoSerializerUpgradeTest extends TestLogger {
 			harness.processElement(value, timestamp++);
 		}
 
-
 		long checkpointId = 1L;
 		long checkpointTimestamp = timestamp + 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/CoordVector.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/CoordVector.java b/flink-tests/src/test/java/org/apache/flink/test/util/CoordVector.java
index 537c61e..aadeaeb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/CoordVector.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/CoordVector.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.test.util;
 
-import java.io.IOException;
-
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.Value;
 
+import java.io.IOException;
+
 /**
  * Implements a feature vector as a multi-dimensional point. Coordinates of that point
  * (= the features) are stored as double values. The distance between two feature vectors is
@@ -31,7 +31,7 @@ import org.apache.flink.types.Value;
  */
 public final class CoordVector implements Value, Comparable<CoordVector> {
 	private static final long serialVersionUID = 1L;
-	
+
 	// coordinate array
 	private double[] coordinates;
 
@@ -44,7 +44,7 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 
 	/**
 	 * Initializes a coordinate vector.
-	 * 
+	 *
 	 * @param coordinates The coordinate vector of a multi-dimensional point.
 	 */
 	public CoordVector(Double[] coordinates) {
@@ -56,7 +56,7 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 
 	/**
 	 * Initializes a coordinate vector.
-	 * 
+	 *
 	 * @param coordinates The coordinate vector of a multi-dimensional point.
 	 */
 	public CoordVector(double[] coordinates) {
@@ -65,16 +65,16 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 
 	/**
 	 * Returns the coordinate vector of a multi-dimensional point.
-	 * 
+	 *
 	 * @return The coordinate vector of a multi-dimensional point.
 	 */
 	public double[] getCoordinates() {
 		return this.coordinates;
 	}
-	
+
 	/**
 	 * Sets the coordinate vector of a multi-dimensional point.
-	 * 
+	 *
 	 * @param coordinates The dimension values of the point.
 	 */
 	public void setCoordinates(double[] coordinates) {
@@ -84,7 +84,7 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 	/**
 	 * Computes the Euclidian distance between this coordinate vector and a
 	 * second coordinate vector.
-	 * 
+	 *
 	 * @param cv The coordinate vector to which the distance is computed.
 	 * @return The Euclidian distance to coordinate vector cv. If cv has a
 	 *         different length than this coordinate vector, -1 is returned.
@@ -98,12 +98,11 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 		double quadSum = 0.0;
 		for (int i = 0; i < this.coordinates.length; i++) {
 			double diff = this.coordinates[i] - cv.coordinates[i];
-			quadSum += diff*diff;
+			quadSum += diff * diff;
 		}
 		return Math.sqrt(quadSum);
 	}
 
-
 	@Override
 	public void read(DataInputView in) throws IOException {
 		int length = in.readInt();
@@ -113,7 +112,6 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 		}
 	}
 
-
 	@Override
 	public void write(DataOutputView out) throws IOException {
 		out.writeInt(this.coordinates.length);
@@ -124,7 +122,7 @@ public final class CoordVector implements Value, Comparable<CoordVector> {
 
 	/**
 	 * Compares this coordinate vector to another key.
-	 * 
+	 *
 	 * @return -1 if the other key is not of type CoordVector. If the other
 	 *         key is also a CoordVector but its length differs from this
 	 *         coordinates vector, -1 is return if this coordinate vector is

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java
index 1409848..dc37c5d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/DataSetUtilsITCase.java
@@ -29,8 +29,9 @@ import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple8;
 import org.apache.flink.api.java.utils.DataSetUtils;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.apache.flink.test.operators.util.CollectionDataSets;
 import org.apache.flink.types.DoubleValue;
+
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -43,6 +44,9 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+/**
+ * Integration tests for {@link DataSetUtils}.
+ */
 @RunWith(Parameterized.class)
 public class DataSetUtilsITCase extends MultipleProgramsTestBase {
 
@@ -52,14 +56,14 @@ public class DataSetUtilsITCase extends MultipleProgramsTestBase {
 
 	@Test
 	public void testCountElementsPerPartition() throws Exception {
-	 	ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-	 	long expectedSize = 100L;
-	 	DataSet<Long> numbers = env.generateSequence(0, expectedSize - 1);
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		long expectedSize = 100L;
+		DataSet<Long> numbers = env.generateSequence(0, expectedSize - 1);
 
-	 	DataSet<Tuple2<Integer, Long>> ds = DataSetUtils.countElementsPerPartition(numbers);
+		DataSet<Tuple2<Integer, Long>> ds = DataSetUtils.countElementsPerPartition(numbers);
 
-	 	Assert.assertEquals(env.getParallelism(), ds.count());
-	 	Assert.assertEquals(expectedSize, ds.sum(1).collect().get(0).f1.longValue());
+		Assert.assertEquals(env.getParallelism(), ds.count());
+		Assert.assertEquals(expectedSize, ds.sum(1).collect().get(0).f1.longValue());
 	}
 
 	@Test
@@ -90,7 +94,7 @@ public class DataSetUtilsITCase extends MultipleProgramsTestBase {
 		long expectedSize = 100L;
 		DataSet<Long> numbers = env.generateSequence(1L, expectedSize);
 
-		DataSet<Long> ids = DataSetUtils.zipWithUniqueId(numbers).map(new MapFunction<Tuple2<Long,Long>, Long>() {
+		DataSet<Long> ids = DataSetUtils.zipWithUniqueId(numbers).map(new MapFunction<Tuple2<Long, Long>, Long>() {
 			@Override
 			public Long map(Tuple2<Long, Long> value) throws Exception {
 				return value.f0;
@@ -118,14 +122,14 @@ public class DataSetUtilsITCase extends MultipleProgramsTestBase {
 		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
 		List<Tuple8<Short, Integer, Long, Float, Double, String, Boolean, DoubleValue>> data = new ArrayList<>();
-		data.add(new Tuple8<>((short)1, 1, 100L, 0.1f, 1.012376, "hello", false, new DoubleValue(50.0)));
-		data.add(new Tuple8<>((short)2, 2, 1000L, 0.2f, 2.003453, "hello", true, new DoubleValue(50.0)));
-		data.add(new Tuple8<>((short)4, 10, 10000L, 0.2f, 75.00005, "null", true, new DoubleValue(50.0)));
-		data.add(new Tuple8<>((short)10, 4, 100L, 0.9f, 79.5, "", true, new DoubleValue(50.0)));
-		data.add(new Tuple8<>((short)5, 5, 1000L, 0.2f, 10.0000001, "a", false, new DoubleValue(50.0)));
-		data.add(new Tuple8<>((short)6, 6, 10L, 0.1f, 0.0000000000023, "", true, new DoubleValue(100.0)));
-		data.add(new Tuple8<>((short)7, 7, 1L, 0.2f, Double.POSITIVE_INFINITY, "abcdefghijklmnop", true, new DoubleValue(100.0)));
-		data.add(new Tuple8<>((short)8, 8, -100L, 0.001f, Double.NaN, "abcdefghi", true, new DoubleValue(100.0)));
+		data.add(new Tuple8<>((short) 1, 1, 100L, 0.1f, 1.012376, "hello", false, new DoubleValue(50.0)));
+		data.add(new Tuple8<>((short) 2, 2, 1000L, 0.2f, 2.003453, "hello", true, new DoubleValue(50.0)));
+		data.add(new Tuple8<>((short) 4, 10, 10000L, 0.2f, 75.00005, "null", true, new DoubleValue(50.0)));
+		data.add(new Tuple8<>((short) 10, 4, 100L, 0.9f, 79.5, "", true, new DoubleValue(50.0)));
+		data.add(new Tuple8<>((short) 5, 5, 1000L, 0.2f, 10.0000001, "a", false, new DoubleValue(50.0)));
+		data.add(new Tuple8<>((short) 6, 6, 10L, 0.1f, 0.0000000000023, "", true, new DoubleValue(100.0)));
+		data.add(new Tuple8<>((short) 7, 7, 1L, 0.2f, Double.POSITIVE_INFINITY, "abcdefghijklmnop", true, new DoubleValue(100.0)));
+		data.add(new Tuple8<>((short) 8, 8, -100L, 0.001f, Double.NaN, "abcdefghi", true, new DoubleValue(100.0)));
 
 		Collections.shuffle(data);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerInputFormat.java
index 8ebb87a..6f6f2a8 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerInputFormat.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerInputFormat.java
@@ -15,11 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.util;
 
-import java.io.IOException;
 import org.apache.flink.api.common.io.GenericInputFormat;
 
+import java.io.IOException;
+
+/**
+ * Generates an infinite series of integer elements with optional read delay.
+ */
 public class InfiniteIntegerInputFormat extends GenericInputFormat<Integer> {
 	private static final long serialVersionUID = 1L;
 	private static final int DELAY = 20;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java
index ad6edd4..660ff08 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/InfiniteIntegerTupleInputFormat.java
@@ -15,12 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.util;
 
-import java.io.IOException;
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
 
+import java.io.IOException;
+
+/**
+ * Generates an infinite series of integer 2-tuples elements with optional read delay.
+ */
 public class InfiniteIntegerTupleInputFormat extends GenericInputFormat<Tuple2<Integer, Integer>> {
 	private static final long serialVersionUID = 1L;
 	private static final int DELAY = 20;

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java
index b0dd78e..6c4546e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/PointFormatter.java
@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.util;
 
-import java.text.DecimalFormat;
-import java.text.DecimalFormatSymbols;
 import org.apache.flink.api.java.io.TextOutputFormat.TextFormatter;
 import org.apache.flink.api.java.tuple.Tuple2;
 
+import java.text.DecimalFormat;
+import java.text.DecimalFormatSymbols;
+
 /**
  * Writes records that contain an id and a CoordVector. The output format is line-based, i.e. one record is written to a
  * line and terminated with '\n'. Within a line the first '|' character separates the id from the CoordVector. The

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java
index 424b781..492499a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/PointInFormat.java
@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.util;
 
+import org.apache.flink.api.common.io.DelimitedInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.flink.api.common.io.DelimitedInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
 
 /**
  * Generates records with an id and a and CoordVector. The input format is line-based, i.e. one record is read from one
@@ -89,7 +91,7 @@ public class PointInFormat extends DelimitedInputFormat<Tuple2<Integer, CoordVec
 		for (int i = 0; i < this.pointValues.length; i++) {
 			this.pointValues[i] = this.dimensionValues.get(i);
 		}
-		
+
 		reuse.f1 = new CoordVector(this.pointValues);
 		return reuse;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
index 4413d3f..1606783 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
@@ -25,8 +25,11 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 import static org.junit.Assert.fail;
 
+/**
+ * Test utilities.
+ */
 public class TestUtils {
-	
+
 	public static JobExecutionResult tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
 		try {
 			return see.execute(name);
@@ -46,7 +49,7 @@ public class TestUtils {
 				}
 			}
 		}
-		
+
 		return null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java b/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java
index c779275..2d34d0d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/UniformIntTupleGeneratorInputFormat.java
@@ -15,14 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.util;
 
-import java.io.IOException;
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.io.GenericInputSplit;
 import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator;
 
+import java.io.IOException;
+
+/**
+ * Generates a series of integer 2-tuples.
+ */
 public class UniformIntTupleGeneratorInputFormat extends GenericInputFormat<Tuple2<Integer, Integer>> {
 	private final int keyTotal;
 	private final int valueTotal;
@@ -38,7 +43,6 @@ public class UniformIntTupleGeneratorInputFormat extends GenericInputFormat<Tupl
 	public void open(GenericInputSplit split) throws IOException {
 		super.open(split);
 		this.generator = new UniformIntTupleGenerator(keyTotal, valueTotal, false);
-
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGenerator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGenerator.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGenerator.java
index fcc45d1..edd3e44 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGenerator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGenerator.java
@@ -19,7 +19,7 @@
 package org.apache.flink.test.windowing.sessionwindows;
 
 /**
- * Basic interface for event generators
+ * Basic interface for event generators.
  *
  * @param <K> session key type
  * @param <E> session event type

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGeneratorFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGeneratorFactory.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGeneratorFactory.java
index 5f91ced..d4c1a47 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGeneratorFactory.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/EventGeneratorFactory.java
@@ -23,9 +23,8 @@ import org.apache.flink.util.Preconditions;
 import java.util.HashMap;
 import java.util.Map;
 
-
 /**
- * Produces the session event generators
+ * Produces the session event generators.
  *
  * @param <K> type of session keys
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorConfiguration.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorConfiguration.java
index acfe544..7591524 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorConfiguration.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorConfiguration.java
@@ -19,7 +19,7 @@
 package org.apache.flink.test.windowing.sessionwindows;
 
 /**
- * Configuration for event generators
+ * Configuration for event generators.
  */
 public final class GeneratorConfiguration {
 
@@ -35,10 +35,11 @@ public final class GeneratorConfiguration {
 	// hint for the maximum additional gap introduced between event times of two generators to separate sessions
 	private final long maxAdditionalSessionGap;
 
-	public GeneratorConfiguration(long allowedLateness,
-	                              int lateEventsWithinLateness,
-	                              int lateEventsAfterLateness,
-	                              long maxAdditionalSessionGap) {
+	public GeneratorConfiguration(
+			long allowedLateness,
+			int lateEventsWithinLateness,
+			int lateEventsAfterLateness,
+			long maxAdditionalSessionGap) {
 		this.allowedLateness = allowedLateness;
 		this.lateEventsWithinLateness = lateEventsWithinLateness;
 		this.lateEventsAfterLateness = lateEventsAfterLateness;
@@ -61,10 +62,11 @@ public final class GeneratorConfiguration {
 		return maxAdditionalSessionGap;
 	}
 
-	public static GeneratorConfiguration of(long allowedLateness,
-	                                        int lateEventsPerSessionWithinLateness,
-	                                        int lateEventsPerSessionOutsideLateness,
-	                                        long maxAdditionalSessionGap) {
+	public static GeneratorConfiguration of(
+			long allowedLateness,
+			int lateEventsPerSessionWithinLateness,
+			int lateEventsPerSessionOutsideLateness,
+			long maxAdditionalSessionGap) {
 		return new GeneratorConfiguration(
 				allowedLateness,
 				lateEventsPerSessionWithinLateness,
@@ -81,4 +83,4 @@ public final class GeneratorConfiguration {
 				", maxAdditionalSessionGap=" + maxAdditionalSessionGap +
 				'}';
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorEventFactory.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorEventFactory.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorEventFactory.java
index 08d5858..0d12e58 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorEventFactory.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/GeneratorEventFactory.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.windowing.sessionwindows;
 /**
  * Factory that produces events of keyed session generators
  *
- * If types of generated events diverge more, we can consider also specify the input parameters to createEvent(...) as
+ * <p>If types of generated events diverge more, we can consider also specify the input parameters to createEvent(...) as
  * a generic object type (containing all the data).
  *
  * @param <K> type of produced event key
@@ -38,10 +38,11 @@ public interface GeneratorEventFactory<K, E> {
 	 * @param timing          indicator for lateness
 	 * @return event for an keyed event generator
 	 */
-	E createEvent(K key,
-	              int sessionId,
-	              int eventId,
-	              long eventTimestamp,
-	              long globalWatermark,
-	              SessionEventGeneratorImpl.Timing timing);
+	E createEvent(
+		K key,
+		int sessionId,
+		int eventId,
+		long eventTimestamp,
+		long globalWatermark,
+		SessionEventGeneratorImpl.Timing timing);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/LongRandomGenerator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/LongRandomGenerator.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/LongRandomGenerator.java
index 132d307..0194959 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/LongRandomGenerator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/LongRandomGenerator.java
@@ -21,6 +21,9 @@ package org.apache.flink.test.windowing.sessionwindows;
 import java.util.Collection;
 import java.util.Random;
 
+/**
+ * Provide additional PRNG methods for selecting in a range and for collection choice.
+ */
 public class LongRandomGenerator extends Random {
 
 	static final long serialVersionUID = 32523525277L;
@@ -60,7 +63,7 @@ public class LongRandomGenerator extends Random {
 	/**
 	 * @return a randomly chosen element from collection
 	 */
-	public <T> T choseRandomElement(Collection<T> collection) {
+	public <T> T chooseRandomElement(Collection<T> collection) {
 		int choice = choseRandomIndex(collection);
 		for (T key : collection) {
 			if (choice == 0) {
@@ -70,4 +73,4 @@ public class LongRandomGenerator extends Random {
 		}
 		return null;
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/ParallelSessionsEventGenerator.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/ParallelSessionsEventGenerator.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/ParallelSessionsEventGenerator.java
index b185f9a..56a95ce 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/ParallelSessionsEventGenerator.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/ParallelSessionsEventGenerator.java
@@ -27,10 +27,10 @@ import java.util.Set;
 /**
  * Generator that employs several (sub-) event generators to produce events for multiple sessions in parallel, i.e.
  * events are emitted in an interleaved way.
- * <p>
- * Even events that belong to different sessions for the same key can be generated in parallel.
- * <p>
- * The watermark is computed as the minimum of watermarks among all current sub-generators.
+ *
+ * <p>Even events that belong to different sessions for the same key can be generated in parallel.
+ *
+ * <p>The watermark is computed as the minimum of watermarks among all current sub-generators.
  *
  * @param <K> session key type
  * @param <E> session event type
@@ -107,7 +107,7 @@ public class ParallelSessionsEventGenerator<K, E> {
 					if (generatorFactory.getProducedGeneratorsCount() < sessionCountLimit) {
 						subGeneratorLists.set(index,
 								generatorFactory.newSessionGeneratorForKey(
-										randomGenerator.choseRandomElement(sessionKeys), getWatermark()));
+										randomGenerator.chooseRandomElement(sessionKeys), getWatermark()));
 					} else {
 						// otherwise removes the sub-generator and shrinks the list of open sessions permanently
 						subGeneratorLists.remove(index);
@@ -140,8 +140,8 @@ public class ParallelSessionsEventGenerator<K, E> {
 	private void initParallelSessionGenerators(int parallelSessions) {
 		for (int i = 0; i < parallelSessions && generatorFactory.getProducedGeneratorsCount() < sessionCountLimit; ++i) {
 			subGeneratorLists.add(generatorFactory.newSessionGeneratorForKey(
-					randomGenerator.choseRandomElement(sessionKeys), 0L));
+					randomGenerator.chooseRandomElement(sessionKeys), 0L));
 		}
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionConfiguration.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionConfiguration.java
index cb481f5..d329a30 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionConfiguration.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionConfiguration.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.windowing.sessionwindows;
 import org.apache.flink.util.Preconditions;
 
 /**
- * Configuration data for a session
+ * Configuration data for a session.
  *
  * @param <K> type of session key
  * @param <E> type of session event
@@ -122,4 +122,4 @@ public final class SessionConfiguration<K, E> {
 				getNumberOfTimelyEvents(),
 				getEventFactory());
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEvent.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEvent.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEvent.java
index 1f5ac60..94bcc2b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEvent.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEvent.java
@@ -36,7 +36,7 @@ public final class SessionEvent<K, V> {
 
 	// event timestamp (in ms)
 	private long eventTimestamp;
-	
+
 	public SessionEvent() {
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEventGeneratorImpl.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEventGeneratorImpl.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEventGeneratorImpl.java
index e8a6df6..34f37a7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEventGeneratorImpl.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionEventGeneratorImpl.java
@@ -33,7 +33,7 @@ import java.util.List;
 public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 
 	/**
-	 * Event timing w.r.t the global watermark
+	 * Event timing w.r.t the global watermark.
 	 */
 	public enum Timing {
 		TIMELY, IN_LATENESS, AFTER_LATENESS
@@ -126,7 +126,7 @@ public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 	}
 
 	/**
-	 * pre-computes and stores the timestamps for timely events in this session in a list (ordered)
+	 * Pre-computes and stores the timestamps for timely events in this session in a list (ordered).
 	 *
 	 * @param minTimestamp              the minimum event time in the session
 	 * @param onTimeEventCountInSession the number of timestamps to generate
@@ -269,7 +269,7 @@ public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 	}
 
 	/**
-	 * internal generator delegate for producing session events that are timely
+	 * Internal generator delegate for producing session events that are timely.
 	 */
 	private class TimelyGenerator extends AbstractEventGenerator {
 
@@ -300,7 +300,7 @@ public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 	}
 
 	/**
-	 * internal generator delegate for producing late session events with timestamps within the allowed lateness
+	 * Internal generator delegate for producing late session events with timestamps within the allowed lateness.
 	 */
 	private class InLatenessGenerator extends AbstractEventGenerator {
 
@@ -331,7 +331,7 @@ public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 	}
 
 	/**
-	 * internal generator delegate for producing late session events with timestamps after the lateness
+	 * Internal generator delegate for producing late session events with timestamps after the lateness.
 	 */
 	private class AfterLatenessGenerator extends AbstractEventGenerator {
 
@@ -361,4 +361,4 @@ public class SessionEventGeneratorImpl<K, E> implements EventGenerator<K, E> {
 		}
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionGeneratorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionGeneratorConfiguration.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionGeneratorConfiguration.java
index 4f29eb3..edd2ba5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionGeneratorConfiguration.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionGeneratorConfiguration.java
@@ -57,4 +57,4 @@ public final class SessionGeneratorConfiguration<K, E> {
 				", generatorConfiguration=" + generatorConfiguration +
 				'}';
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
index 4e4cb20..494b8d6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
@@ -34,6 +34,7 @@ import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -44,7 +45,7 @@ import java.util.List;
 import java.util.Set;
 
 /**
- * ITCase for Session Windows
+ * ITCase for Session Windows.
  */
 public class SessionWindowITCase extends StreamingMultipleProgramsTestBase {
 
@@ -100,11 +101,10 @@ public class SessionWindowITCase extends StreamingMultipleProgramsTestBase {
 			WindowFunction<SessionEvent<Integer, TestEventPayload>,
 					String, Tuple, TimeWindow> windowFunction) throws Exception {
 
-
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		WindowedStream<SessionEvent<Integer, TestEventPayload>, Tuple, TimeWindow> windowedStream
-				= env.addSource(dataSource).keyBy("sessionKey")
+		WindowedStream<SessionEvent<Integer, TestEventPayload>, Tuple, TimeWindow> windowedStream =
+				env.addSource(dataSource).keyBy("sessionKey")
 				.window(EventTimeSessionWindows.withGap(Time.milliseconds(MAX_SESSION_EVENT_GAP_MS)));
 
 		if (ALLOWED_LATENESS_MS != Long.MAX_VALUE) {
@@ -129,7 +129,7 @@ public class SessionWindowITCase extends StreamingMultipleProgramsTestBase {
 	}
 
 	/**
-	 * Window function that performs correctness checks for this test case
+	 * Window function that performs correctness checks for this test case.
 	 */
 	private static final class ValidatingWindowFunction extends RichWindowFunction<SessionEvent<Integer,
 			TestEventPayload>, String, Tuple, TimeWindow> {
@@ -200,7 +200,7 @@ public class SessionWindowITCase extends StreamingMultipleProgramsTestBase {
 	}
 
 	/**
-	 * A data source that is fed from a ParallelSessionsEventGenerator
+	 * A data source that is fed from a ParallelSessionsEventGenerator.
 	 */
 	private static final class SessionEventGeneratorDataSource
 			implements SourceFunction<SessionEvent<Integer, TestEventPayload>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/TestEventPayload.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/TestEventPayload.java b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/TestEventPayload.java
index 13b1180..c532269 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/TestEventPayload.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/TestEventPayload.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.windowing.sessionwindows;
 import org.apache.flink.util.Preconditions;
 
 /**
- * Test payload that contains useful information for the correctness checks in our test
+ * Test payload that contains useful information for the correctness checks in our test.
  */
 public final class TestEventPayload {
 
@@ -103,4 +103,4 @@ public final class TestEventPayload {
 		return new TestEventPayload(watermark, sessionId, eventId, timing);
 	}
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/9bd491e0/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala
index 2dabb56..9d286fe 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.operators.Order
 import org.apache.flink.api.java.io.DiscardingOutputFormat
 import org.apache.flink.api.scala._
 import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.test.javaApiOperators.GroupCombineITCase.ScalaGroupCombineFunctionExample
+import org.apache.flink.test.operators.GroupCombineITCase.ScalaGroupCombineFunctionExample
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
 import org.apache.flink.test.util.MultipleProgramsTestBase
 import org.apache.flink.util.Collector