You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tz...@apache.org on 2017/06/07 17:22:48 UTC

[01/12] flink git commit: [hotfix] [tests] Add serialVersionUID to InterruptLockingStateHandle in tests

Repository: flink
Updated Branches:
  refs/heads/release-1.3 977495856 -> d4a646a03


[hotfix] [tests] Add serialVersionUID to InterruptLockingStateHandle in tests


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

Branch: refs/heads/release-1.3
Commit: 53e69b490dbbf728c192c1ff526a6fcca1cf1426
Parents: 9774958
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Wed Jun 7 16:41:02 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:51:24 2017 +0200

----------------------------------------------------------------------
 .../streaming/runtime/tasks/InterruptSensitiveRestoreTest.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/53e69b49/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index 4435247..30bb3f5 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -281,6 +281,8 @@ public class InterruptSensitiveRestoreTest {
 	@SuppressWarnings("serial")
 	private static class InterruptLockingStateHandle implements StreamStateHandle {
 
+		private static final long serialVersionUID = 1L;
+
 		private volatile boolean closed;
 
 		@Override


[10/12] flink git commit: [FLINK-6830] [DataStream] Port window operator migration tests for Flink 1.3

Posted by tz...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
new file mode 100644
index 0000000..ef6bd3e
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
@@ -0,0 +1,1087 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import static org.junit.Assert.fail;
+
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ReducingStateDescriptor;
+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.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType;
+import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
+import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+import org.apache.flink.util.Collector;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Tests for checking whether {@link WindowOperator} can restore from snapshots that were done
+ * using previous Flink versions' {@link WindowOperator}.
+ *
+ * <p>This also checks whether {@link WindowOperator} can restore from a checkpoint of the
+ * aligned processing-time windows operator of previous Flink versions.
+ *
+ * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
+ * the corresponding Flink release-* branch.
+ */
+@RunWith(Parameterized.class)
+public class WindowOperatorMigrationTest {
+
+	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
+	public static Collection<MigrationVersion> parameters () {
+		return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3);
+	}
+
+	/**
+	 * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+	 * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints
+	 */
+	private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+	private final MigrationVersion testMigrateVersion;
+
+	public WindowOperatorMigrationTest(MigrationVersion testMigrateVersion) {
+		this.testMigrateVersion = testMigrateVersion;
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeSessionWindowsWithCountTriggerSnapshot() throws Exception {
+		final int SESSION_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
+				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
+				PurgingTrigger.of(CountTrigger.of(4)),
+				0,
+				null /* late data output tag */);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
+
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreSessionWindowsWithCountTrigger() throws Exception {
+
+		final int SESSION_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
+				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
+				PurgingTrigger.of(CountTrigger.of(4)),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-session-with-stateful-trigger-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
+
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
+
+		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
+
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeSessionWindowsWithCountTriggerInMintConditionSnapshot() throws Exception {
+
+		final int SESSION_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
+				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
+				PurgingTrigger.of(CountTrigger.of(4)),
+				0,
+				null /* late data output tag */);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+	}
+
+	/**
+	 * This checks that we can restore from a virgin {@code WindowOperator} that has never seen
+	 * any elements.
+	 */
+	@Test
+	public void testRestoreSessionWindowsWithCountTriggerInMintCondition() throws Exception {
+
+		final int SESSION_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
+				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
+				PurgingTrigger.of(CountTrigger.of(4)),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-session-with-stateful-trigger-mint-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-10", 0L, 6500L), 6499));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
+
+		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
+
+		testHarness.close();
+	}
+
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeReducingEventTimeWindowsSnapshot() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				EventTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+
+		testHarness.processWatermark(new Watermark(999));
+		expectedOutput.add(new Watermark(999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(1999));
+		expectedOutput.add(new Watermark(1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-reduce-event-time-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreReducingEventTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				EventTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-reduce-event-time-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.processWatermark(new Watermark(2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new Watermark(2999));
+
+		testHarness.processWatermark(new Watermark(3999));
+		expectedOutput.add(new Watermark(3999));
+
+		testHarness.processWatermark(new Watermark(4999));
+		expectedOutput.add(new Watermark(4999));
+
+		testHarness.processWatermark(new Watermark(5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
+		expectedOutput.add(new Watermark(5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeApplyEventTimeWindowsSnapshot() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
+				EventTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
+
+		testHarness.processWatermark(new Watermark(999));
+		expectedOutput.add(new Watermark(999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(1999));
+		expectedOutput.add(new Watermark(1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-apply-event-time-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreApplyEventTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
+				EventTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-apply-event-time-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.processWatermark(new Watermark(2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new Watermark(2999));
+
+		testHarness.processWatermark(new Watermark(3999));
+		expectedOutput.add(new Watermark(3999));
+
+		testHarness.processWatermark(new Watermark(4999));
+		expectedOutput.add(new Watermark(4999));
+
+		testHarness.processWatermark(new Watermark(5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
+		expectedOutput.add(new Watermark(5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeReducingProcessingTimeWindowsSnapshot() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(10);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
+
+		testHarness.setProcessingTime(3010);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-reduce-processing-time-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+
+	}
+
+	@Test
+	public void testRestoreReducingProcessingTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-reduce-processing-time-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(3020);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
+
+		testHarness.setProcessingTime(6000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeApplyProcessingTimeWindowsSnapshot() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(10);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
+
+		testHarness.setProcessingTime(3010);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		// do snapshot and save to file
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-apply-processing-time-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreApplyProcessingTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-apply-processing-time-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(3020);
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
+
+		testHarness.setProcessingTime(6000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeAggregatingAlignedProcessingTimeWindowsSnapshot() throws Exception {
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		AggregatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>> operator =
+			new AggregatingProcessingTimeWindowOperator<>(
+				new ReduceFunction<Tuple2<String, Integer>>() {
+					private static final long serialVersionUID = -8913160567151867987L;
+
+					@Override
+					public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
+						return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
+					}
+				},
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				inputType.createSerializer(new ExecutionConfig()),
+				3000,
+				3000);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(3);
+
+		// timestamp is ignored in processing time
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+
+		// do a snapshot, close and restore again
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-aggr-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreAggregatingAlignedProcessingTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */,
+				LegacyWindowOperatorType.FAST_AGGREGATING);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-aggr-aligned-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(5000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+
+		testHarness.setProcessingTime(7000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.close();
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeAlignedProcessingTimeWindowsSnapshot() throws Exception {
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		AccumulatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>> operator =
+			new AccumulatingProcessingTimeWindowOperator<>(
+					new InternalIterableWindowFunction<>(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() {
+
+						private static final long serialVersionUID = 6551516443265733803L;
+
+						@Override
+						public void apply(String s, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<Tuple2<String, Integer>> out) throws Exception {
+							int sum = 0;
+							for (Tuple2<String, Integer> anInput : input) {
+								sum += anInput.f1;
+							}
+							out.collect(new Tuple2<>(s, sum));
+						}
+					}),
+					new TupleKeySelector(),
+					BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+					inputType.createSerializer(new ExecutionConfig()),
+					3000,
+					3000);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(3);
+
+		// timestamp is ignored in processing time
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+
+		// do a snapshot, close and restore again
+		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
+		OperatorSnapshotUtil.writeStateHandle(
+			snapshot,
+			"src/test/resources/win-op-migration-test-accum-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestoreAccumulatingAlignedProcessingTimeWindows() throws Exception {
+		final int WINDOW_SIZE = 3;
+
+		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+
+		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
+				new SumReducer(),
+				inputType.createSerializer(new ExecutionConfig()));
+
+		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
+				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
+				new TimeWindow.Serializer(),
+				new TupleKeySelector(),
+				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
+				stateDesc,
+				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
+				ProcessingTimeTrigger.create(),
+				0,
+				null /* late data output tag */,
+				LegacyWindowOperatorType.FAST_ACCUMULATING);
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"win-op-migration-test-accum-aligned-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		testHarness.setProcessingTime(5000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
+
+		testHarness.setProcessingTime(7000);
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
+
+		testHarness.close();
+	}
+
+
+	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple2<String, Integer> value) throws Exception {
+			return value.f0;
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	private static class Tuple2ResultSortComparator implements Comparator<Object> {
+		@Override
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Watermark || o2 instanceof Watermark) {
+				return 0;
+			} else {
+				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
+				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
+				if (sr0.getTimestamp() != sr1.getTimestamp()) {
+					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+				}
+				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
+				if (comparison != 0) {
+					return comparison;
+				} else {
+					return sr0.getValue().f1 - sr1.getValue().f1;
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	private static class Tuple3ResultSortComparator implements Comparator<Object> {
+		@Override
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Watermark || o2 instanceof Watermark) {
+				return 0;
+			} else {
+				StreamRecord<Tuple3<String, Long, Long>> sr0 = (StreamRecord<Tuple3<String, Long, Long>>) o1;
+				StreamRecord<Tuple3<String, Long, Long>> sr1 = (StreamRecord<Tuple3<String, Long, Long>>) o2;
+				if (sr0.getTimestamp() != sr1.getTimestamp()) {
+					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+				}
+				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
+				if (comparison != 0) {
+					return comparison;
+				} else {
+					comparison = (int) (sr0.getValue().f1 - sr1.getValue().f1);
+					if (comparison != 0) {
+						return comparison;
+					}
+					return (int) (sr0.getValue().f1 - sr1.getValue().f1);
+				}
+			}
+		}
+	}
+
+	public static class SumReducer implements ReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
+				Tuple2<String, Integer> value2) throws Exception {
+			return new Tuple2<>(value2.f0, value1.f1 + value2.f1);
+		}
+	}
+
+	public static class RichSumReducer<W extends Window> extends RichWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, W> {
+		private static final long serialVersionUID = 1L;
+
+		private boolean openCalled = false;
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+			openCalled = true;
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+		}
+
+		@Override
+		public void apply(String key,
+				W window,
+				Iterable<Tuple2<String, Integer>> input,
+				Collector<Tuple2<String, Integer>> out) throws Exception {
+
+			if (!openCalled) {
+				fail("Open was not called");
+			}
+			int sum = 0;
+
+			for (Tuple2<String, Integer> t: input) {
+				sum += t.f1;
+			}
+			out.collect(new Tuple2<>(key, sum));
+
+		}
+
+	}
+
+	public static class SessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void apply(String key,
+				TimeWindow window,
+				Iterable<Tuple2<String, Integer>> values,
+				Collector<Tuple3<String, Long, Long>> out) throws Exception {
+			int sum = 0;
+			for (Tuple2<String, Integer> i: values) {
+				sum += i.f1;
+			}
+			String resultString = key + "-" + sum;
+			out.collect(new Tuple3<>(resultString, window.getStart(), window.getEnd()));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot
new file mode 100644
index 0000000..f85adf7
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot
new file mode 100644
index 0000000..7ed2ab9
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-aggr-aligned-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot
new file mode 100644
index 0000000..be4b306
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-event-time-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot
new file mode 100644
index 0000000..a642a3a
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-apply-processing-time-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot
new file mode 100644
index 0000000..a9f0075
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-event-time-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot
new file mode 100644
index 0000000..7610073
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-reduce-processing-time-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot
new file mode 100644
index 0000000..8ad2d39
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot
new file mode 100644
index 0000000..7337bae
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.3-snapshot differ


[11/12] flink git commit: [FLINK-6830] [DataStream] Port window operator migration tests for Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [DataStream] Port window operator migration tests for Flink 1.3

This commit also consolidates all Flink 1.1 and 1.2 window operator
migration tests to a single WindowOperatorMigrationTest class.
Parameterization is used to test restoring from different previous Flink
version snapshots.


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

Branch: refs/heads/release-1.3
Commit: e5a435b2ddb555f625a419897ed56f792f7372b3
Parents: 74bb9a8
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sun Jun 4 00:55:48 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 19:09:25 2017 +0200

----------------------------------------------------------------------
 .../WindowOperatorFrom11MigrationTest.java      |  896 ---------------
 .../WindowOperatorFrom12MigrationTest.java      | 1014 ----------------
 .../windowing/WindowOperatorMigrationTest.java  | 1087 ++++++++++++++++++
 ...gration-test-accum-aligned-flink1.3-snapshot |  Bin 0 -> 222 bytes
 ...igration-test-aggr-aligned-flink1.3-snapshot |  Bin 0 -> 187 bytes
 ...tion-test-apply-event-time-flink1.3-snapshot |  Bin 0 -> 4629 bytes
 ...test-apply-processing-time-flink1.3-snapshot |  Bin 0 -> 4521 bytes
 ...ion-test-reduce-event-time-flink1.3-snapshot |  Bin 0 -> 3496 bytes
 ...est-reduce-processing-time-flink1.3-snapshot |  Bin 0 -> 3437 bytes
 ...sion-with-stateful-trigger-flink1.3-snapshot |  Bin 0 -> 8679 bytes
 ...with-stateful-trigger-mint-flink1.3-snapshot |  Bin 0 -> 7650 bytes
 11 files changed, 1087 insertions(+), 1910 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom11MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom11MigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom11MigrationTest.java
deleted file mode 100644
index 9ec1923..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom11MigrationTest.java
+++ /dev/null
@@ -1,896 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-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.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType;
-import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-import java.net.URL;
-import java.util.Comparator;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.fail;
-
-/**
- * Tests for checking whether {@link WindowOperator} can restore from snapshots that were done
- * using the Flink 1.1 {@link WindowOperator}.
- *
- * <p>
- * This also checks whether {@link WindowOperator} can restore from a checkpoint of the Flink 1.1
- * aligned processing-time windows operator.
- *
- * <p>For regenerating the binary snapshot file you have to run the commented out portion
- * of each test on a checkout of the Flink 1.1 branch.
- */
-public class WindowOperatorFrom11MigrationTest {
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = WindowOperatorFrom11MigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreSessionWindowsWithCountTriggerFromFlink11() throws Exception {
-
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0, 0);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.1-snapshot");
-		testHarness.close();
-        */
-
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-session-with-stateful-trigger-flink1.1-snapshot"));
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
-
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	/**
-	 * This checks that we can restore from a virgin {@code WindowOperator} that has never seen
-	 * any elements.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreSessionWindowsWithCountTriggerInMintConditionFromFlink11() throws Exception {
-
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0, 0);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.1-snapshot");
-		testHarness.close();
-		*/
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-session-with-stateful-trigger-mint-flink1.1-snapshot"));
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-10", 0L, 6500L), 6499));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreReducingEventTimeWindowsFromFlink11() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
-
-		testHarness.processWatermark(new Watermark(999));
-		expectedOutput.add(new Watermark(999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processWatermark(new Watermark(1999));
-		expectedOutput.add(new Watermark(1999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-reduce-event-time-flink1.1-snapshot");
-		testHarness.close();
-
-		*/
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-reduce-event-time-flink1.1-snapshot"));
-		testHarness.open();
-
-		testHarness.processWatermark(new Watermark(2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new Watermark(2999));
-
-		testHarness.processWatermark(new Watermark(3999));
-		expectedOutput.add(new Watermark(3999));
-
-		testHarness.processWatermark(new Watermark(4999));
-		expectedOutput.add(new Watermark(4999));
-
-		testHarness.processWatermark(new Watermark(5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
-		expectedOutput.add(new Watermark(5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreApplyEventTimeWindowsFromFlink11() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
-
-		testHarness.processWatermark(new Watermark(999));
-		expectedOutput.add(new Watermark(999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processWatermark(new Watermark(1999));
-		expectedOutput.add(new Watermark(1999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-apply-event-time-flink1.1-snapshot");
-		testHarness.close();
-
-		*/
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-apply-event-time-flink1.1-snapshot"));
-		testHarness.open();
-
-		testHarness.processWatermark(new Watermark(2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new Watermark(2999));
-
-		testHarness.processWatermark(new Watermark(3999));
-		expectedOutput.add(new Watermark(3999));
-
-		testHarness.processWatermark(new Watermark(4999));
-		expectedOutput.add(new Watermark(4999));
-
-		testHarness.processWatermark(new Watermark(5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
-		expectedOutput.add(new Watermark(5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreReducingProcessingTimeWindowsFromFlink11() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		TestTimeServiceProvider timeServiceProvider = new TestTimeServiceProvider();
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), timeServiceProvider);
-
-		testHarness.configureForKeyedStream(new WindowOperatorTest.TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		timeServiceProvider.setCurrentTime(10);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
-
-		timeServiceProvider.setCurrentTime(3010);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-reduce-processing-time-flink1.1-snapshot");
-		testHarness.close();
-		*/
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-reduce-processing-time-flink1.1-snapshot"));
-		testHarness.open();
-
-		testHarness.setProcessingTime(3020);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
-
-		testHarness.setProcessingTime(6000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRestoreApplyProcessingTimeWindowsFromFlink11() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		/*
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
-		TestTimeServiceProvider timeServiceProvider = new TestTimeServiceProvider();
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new OneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), timeServiceProvider);
-
-		testHarness.configureForKeyedStream(new WindowOperatorTest.TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		timeServiceProvider.setCurrentTime(10);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
-
-		timeServiceProvider.setCurrentTime(3010);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new WindowOperatorTest.Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-apply-processing-time-flink1.1-snapshot");
-		testHarness.close();
-		*/
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename(
-				"win-op-migration-test-apply-processing-time-flink1.1-snapshot"));
-		testHarness.open();
-
-		testHarness.setProcessingTime(3020);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
-
-		testHarness.setProcessingTime(6000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAggregatingAlignedProcessingTimeWindowsFromFlink11() throws Exception {
-		/*
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AggregatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>> operator =
-			new AggregatingProcessingTimeWindowOperator<>(
-				new ReduceFunction<Tuple2<String, Integer>>() {
-					private static final long serialVersionUID = -8913160567151867987L;
-
-					@Override
-					public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
-						return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-					}
-				},
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				inputType.createSerializer(new ExecutionConfig()),
-				3000,
-				3000);
-
-		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new OneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		testTimeProvider.setCurrentTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-aggr-aligned-flink1.1-snapshot");
-		testHarness.close();
-
-		*/
-
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_AGGREGATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint("src/test/resources/win-op-migration-test-aggr-aligned-flink1.1-snapshot");
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAccumulatingAlignedProcessingTimeWindowsFromFlink11() throws Exception {
-		/*
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AccumulatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>> operator =
-			new AccumulatingProcessingTimeWindowOperator<>(
-				new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() {
-
-					private static final long serialVersionUID = 6551516443265733803L;
-
-					@Override
-					public void apply(String s, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<Tuple2<String, Integer>> out) throws Exception {
-						int sum = 0;
-						for (Tuple2<String, Integer> anInput : input) {
-							sum += anInput.f1;
-						}
-						out.collect(new Tuple2<>(s, sum));
-					}
-				},
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				inputType.createSerializer(new ExecutionConfig()),
-				3000,
-				3000);
-
-		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-			new OneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider);
-
-		testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		testTimeProvider.setCurrentTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/win-op-migration-test-accum-aligned-flink1.1-snapshot");
-		testHarness.close();
-
-		*/
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_ACCUMULATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint("src/test/resources/win-op-migration-test-accum-aligned-flink1.1-snapshot");
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-
-	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple2<String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	private static class Tuple2ResultSortComparator implements Comparator<Object> {
-		@Override
-		public int compare(Object o1, Object o2) {
-			if (o1 instanceof Watermark || o2 instanceof Watermark) {
-				return 0;
-			} else {
-				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
-				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
-				if (sr0.getTimestamp() != sr1.getTimestamp()) {
-					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
-				}
-				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
-				if (comparison != 0) {
-					return comparison;
-				} else {
-					return sr0.getValue().f1 - sr1.getValue().f1;
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	private static class Tuple3ResultSortComparator implements Comparator<Object> {
-		@Override
-		public int compare(Object o1, Object o2) {
-			if (o1 instanceof Watermark || o2 instanceof Watermark) {
-				return 0;
-			} else {
-				StreamRecord<Tuple3<String, Long, Long>> sr0 = (StreamRecord<Tuple3<String, Long, Long>>) o1;
-				StreamRecord<Tuple3<String, Long, Long>> sr1 = (StreamRecord<Tuple3<String, Long, Long>>) o2;
-				if (sr0.getTimestamp() != sr1.getTimestamp()) {
-					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
-				}
-				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
-				if (comparison != 0) {
-					return comparison;
-				} else {
-					comparison = (int) (sr0.getValue().f1 - sr1.getValue().f1);
-					if (comparison != 0) {
-						return comparison;
-					}
-					return (int) (sr0.getValue().f2 - sr1.getValue().f2);
-				}
-			}
-		}
-	}
-
-	public static class SumReducer implements ReduceFunction<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
-				Tuple2<String, Integer> value2) throws Exception {
-			return new Tuple2<>(value2.f0, value1.f1 + value2.f1);
-		}
-	}
-
-	public static class RichSumReducer<W extends Window> extends RichWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, W> {
-		private static final long serialVersionUID = 1L;
-
-		private boolean openCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-		}
-
-		@Override
-		public void apply(String key,
-				W window,
-				Iterable<Tuple2<String, Integer>> input,
-				Collector<Tuple2<String, Integer>> out) throws Exception {
-
-			if (!openCalled) {
-				fail("Open was not called");
-			}
-			int sum = 0;
-
-			for (Tuple2<String, Integer> t: input) {
-				sum += t.f1;
-			}
-			out.collect(new Tuple2<>(key, sum));
-
-		}
-
-	}
-
-	public static class SessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void apply(String key,
-				TimeWindow window,
-				Iterable<Tuple2<String, Integer>> values,
-				Collector<Tuple3<String, Long, Long>> out) throws Exception {
-			int sum = 0;
-			for (Tuple2<String, Integer> i: values) {
-				sum += i.f1;
-			}
-			String resultString = key + "-" + sum;
-			out.collect(new Tuple3<>(resultString, window.getStart(), window.getEnd()));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/e5a435b2/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom12MigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom12MigrationTest.java
deleted file mode 100644
index 0d3a6dc..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom12MigrationTest.java
+++ /dev/null
@@ -1,1014 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import static org.junit.Assert.fail;
-
-import java.util.Comparator;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-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.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType;
-import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.assigners.EventTimeSessionWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.util.Collector;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * Tests for checking whether {@link WindowOperator} can restore from snapshots that were done
- * using the Flink 1.2 {@link WindowOperator}.
- *
- * <p>This also checks whether {@link WindowOperator} can restore from a checkpoint of the Flink 1.2
- * aligned processing-time windows operator.
- *
- * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
- * the Flink 1.2 branch.
- */
-public class WindowOperatorFrom12MigrationTest {
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeSessionWindowsWithCountTriggerSnapshot() throws Exception {
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
-
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-session-with-stateful-trigger-flink1.2-snapshot");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreSessionWindowsWithCountTrigger() throws Exception {
-
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-session-with-stateful-trigger-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
-
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeSessionWindowsWithCountTriggerInMintConditionSnapshot() throws Exception {
-
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-session-with-stateful-trigger-mint-flink1.2-snapshot");
-
-		testHarness.close();
-	}
-
-	/**
-	 * This checks that we can restore from a virgin {@code WindowOperator} that has never seen
-	 * any elements.
-	 */
-	@Test
-	public void testRestoreSessionWindowsWithCountTriggerInMintCondition() throws Exception {
-
-		final int SESSION_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple3<String, Long, Long>, TimeWindow> operator = new WindowOperator<>(
-				EventTimeSessionWindows.withGap(Time.seconds(SESSION_SIZE)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new SessionWindowFunction()),
-				PurgingTrigger.of(CountTrigger.of(4)),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-session-with-stateful-trigger-mint-flink1.2-snapshot")));
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 2), 1000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3), 2500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 4), 3500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 10));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 1000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 2500));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 6000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 2), 6500));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3), 7000));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key2-10", 0L, 6500L), 6499));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		// add an element that merges the two "key1" sessions, they should now have count 6, and therfore fire
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 10), 4500));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple3<>("key1-22", 10L, 10000L), 9999L));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator());
-
-		testHarness.close();
-	}
-
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeReducingEventTimeWindowsSnapshot() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
-
-		testHarness.processWatermark(new Watermark(999));
-		expectedOutput.add(new Watermark(999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processWatermark(new Watermark(1999));
-		expectedOutput.add(new Watermark(1999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-reduce-event-time-flink1.2-snapshot");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreReducingEventTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-reduce-event-time-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.processWatermark(new Watermark(2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new Watermark(2999));
-
-		testHarness.processWatermark(new Watermark(3999));
-		expectedOutput.add(new Watermark(3999));
-
-		testHarness.processWatermark(new Watermark(4999));
-		expectedOutput.add(new Watermark(4999));
-
-		testHarness.processWatermark(new Watermark(5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
-		expectedOutput.add(new Watermark(5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeApplyEventTimeWindowsSnapshot() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		// add elements out-of-order
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 3000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 0));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 999));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1998));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 1000));
-
-		testHarness.processWatermark(new Watermark(999));
-		expectedOutput.add(new Watermark(999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processWatermark(new Watermark(1999));
-		expectedOutput.add(new Watermark(1999));
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-apply-event-time-flink1.2-snapshot");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreApplyEventTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				EventTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-apply-event-time-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.processWatermark(new Watermark(2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new Watermark(2999));
-
-		testHarness.processWatermark(new Watermark(3999));
-		expectedOutput.add(new Watermark(3999));
-
-		testHarness.processWatermark(new Watermark(4999));
-		expectedOutput.add(new Watermark(4999));
-
-		testHarness.processWatermark(new Watermark(5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), 5999));
-		expectedOutput.add(new Watermark(5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeReducingProcessingTimeWindowsSnapshot() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.setProcessingTime(10);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
-
-		testHarness.setProcessingTime(3010);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-reduce-processing-time-flink1.2-snapshot");
-
-		testHarness.close();
-
-	}
-
-	@Test
-	public void testRestoreReducingProcessingTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-reduce-processing-time-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.setProcessingTime(3020);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
-
-		testHarness.setProcessingTime(6000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeApplyProcessingTimeWindowsSnapshot() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.setProcessingTime(10);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1)));
-
-		testHarness.setProcessingTime(3010);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key3", 1)));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 1), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 1), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		// do snapshot and save to file
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-apply-processing-time-flink1.2-snapshot");
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreApplyProcessingTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ListStateDescriptor<Tuple2<String, Integer>> stateDesc = new ListStateDescriptor<>("window-contents",
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Iterable<Tuple2<String, Integer>>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-apply-processing-time-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.setProcessingTime(3020);
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 3)));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 3)));
-
-		testHarness.setProcessingTime(6000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), 5999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key3", 1), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeAggregatingAlignedProcessingTimeWindowsSnapshot() throws Exception {
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AggregatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>> operator =
-			new AggregatingProcessingTimeWindowOperator<>(
-				new ReduceFunction<Tuple2<String, Integer>>() {
-					private static final long serialVersionUID = -8913160567151867987L;
-
-					@Override
-					public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
-						return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-					}
-				},
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				inputType.createSerializer(new ExecutionConfig()),
-				3000,
-				3000);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-aggr-aligned-flink1.2-snapshot");
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAggregatingAlignedProcessingTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_AGGREGATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-aggr-aligned-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeAlignedProcessingTimeWindowsSnapshot() throws Exception {
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AccumulatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>> operator =
-			new AccumulatingProcessingTimeWindowOperator<>(
-					new InternalIterableWindowFunction<>(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() {
-
-						private static final long serialVersionUID = 6551516443265733803L;
-
-						@Override
-						public void apply(String s, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<Tuple2<String, Integer>> out) throws Exception {
-							int sum = 0;
-							for (Tuple2<String, Integer> anInput : input) {
-								sum += anInput.f1;
-							}
-							out.collect(new Tuple2<>(s, sum));
-						}
-					}),
-					new TupleKeySelector(),
-					BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-					inputType.createSerializer(new ExecutionConfig()),
-					3000,
-					3000);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/win-op-migration-test-accum-aligned-flink1.2-snapshot");
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAccumulatingAlignedProcessingTimeWindows() throws Exception {
-		final int WINDOW_SIZE = 3;
-
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
-				new SumReducer(),
-				inputType.createSerializer(new ExecutionConfig()));
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingProcessingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_ACCUMULATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("win-op-migration-test-accum-aligned-flink1.2-snapshot")));
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-
-	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple2<String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	private static class Tuple2ResultSortComparator implements Comparator<Object> {
-		@Override
-		public int compare(Object o1, Object o2) {
-			if (o1 instanceof Watermark || o2 instanceof Watermark) {
-				return 0;
-			} else {
-				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
-				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
-				if (sr0.getTimestamp() != sr1.getTimestamp()) {
-					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
-				}
-				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
-				if (comparison != 0) {
-					return comparison;
-				} else {
-					return sr0.getValue().f1 - sr1.getValue().f1;
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	private static class Tuple3ResultSortComparator implements Comparator<Object> {
-		@Override
-		public int compare(Object o1, Object o2) {
-			if (o1 instanceof Watermark || o2 instanceof Watermark) {
-				return 0;
-			} else {
-				StreamRecord<Tuple3<String, Long, Long>> sr0 = (StreamRecord<Tuple3<String, Long, Long>>) o1;
-				StreamRecord<Tuple3<String, Long, Long>> sr1 = (StreamRecord<Tuple3<String, Long, Long>>) o2;
-				if (sr0.getTimestamp() != sr1.getTimestamp()) {
-					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
-				}
-				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
-				if (comparison != 0) {
-					return comparison;
-				} else {
-					comparison = (int) (sr0.getValue().f1 - sr1.getValue().f1);
-					if (comparison != 0) {
-						return comparison;
-					}
-					return (int) (sr0.getValue().f1 - sr1.getValue().f1);
-				}
-			}
-		}
-	}
-
-	public static class SumReducer implements ReduceFunction<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-		@Override
-		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
-				Tuple2<String, Integer> value2) throws Exception {
-			return new Tuple2<>(value2.f0, value1.f1 + value2.f1);
-		}
-	}
-
-	public static class RichSumReducer<W extends Window> extends RichWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, W> {
-		private static final long serialVersionUID = 1L;
-
-		private boolean openCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-		}
-
-		@Override
-		public void apply(String key,
-				W window,
-				Iterable<Tuple2<String, Integer>> input,
-				Collector<Tuple2<String, Integer>> out) throws Exception {
-
-			if (!openCalled) {
-				fail("Open was not called");
-			}
-			int sum = 0;
-
-			for (Tuple2<String, Integer> t: input) {
-				sum += t.f1;
-			}
-			out.collect(new Tuple2<>(key, sum));
-
-		}
-
-	}
-
-	public static class SessionWindowFunction implements WindowFunction<Tuple2<String, Integer>, Tuple3<String, Long, Long>, String, TimeWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void apply(String key,
-				TimeWindow window,
-				Iterable<Tuple2<String, Integer>> values,
-				Collector<Tuple3<String, Long, Long>> out) throws Exception {
-			int sum = 0;
-			for (Tuple2<String, Integer> i: values) {
-				sum += i.f1;
-			}
-			String resultString = key + "-" + sum;
-			out.collect(new Tuple3<>(resultString, window.getStart(), window.getEnd()));
-		}
-	}
-}


[02/12] flink git commit: [FLINK-6844] [scala] Implement compatibility methods for TraversableSerializer

Posted by tz...@apache.org.
[FLINK-6844] [scala] Implement compatibility methods for TraversableSerializer

This closes #4081.


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

Branch: refs/heads/release-1.3
Commit: e1e207c898ed436df656d01364cf0e5fa818b730
Parents: 53e69b4
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Mon Jun 5 20:52:57 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:51:51 2017 +0200

----------------------------------------------------------------------
 .../scala/typeutils/TraversableSerializer.scala | 41 ++++++++++++++++++--
 1 file changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e1e207c8/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
index 1ac46f9..6299a24 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.typeutils
 import java.io.ObjectInputStream
 
 import org.apache.flink.annotation.Internal
-import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializer, TypeSerializerConfigSnapshot}
+import org.apache.flink.api.common.typeutils._
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 
 import scala.collection.generic.CanBuildFrom
@@ -152,11 +152,46 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E](
   }
 
   override def snapshotConfiguration(): TypeSerializerConfigSnapshot = {
-    throw new UnsupportedOperationException()
+    new TraversableSerializer.TraversableSerializerConfigSnapshot[E](elementSerializer)
   }
 
   override def ensureCompatibility(
       configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[T] = {
-    throw new UnsupportedOperationException()
+
+    configSnapshot match {
+      case traversableSerializerConfigSnapshot:
+          TraversableSerializer.TraversableSerializerConfigSnapshot[E] =>
+
+        val elemCompatRes = CompatibilityUtil.resolveCompatibilityResult(
+          traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0,
+          classOf[UnloadableDummyTypeSerializer[_]],
+          traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f1,
+          elementSerializer)
+
+        if (elemCompatRes.isRequiresMigration) {
+          CompatibilityResult.requiresMigration()
+        } else {
+          CompatibilityResult.compatible()
+        }
+
+      case _ => CompatibilityResult.requiresMigration()
+    }
+  }
+}
+
+object TraversableSerializer {
+
+  class TraversableSerializerConfigSnapshot[E](
+      private var elementSerializer: TypeSerializer[E])
+    extends CompositeTypeSerializerConfigSnapshot(elementSerializer) {
+
+    /** This empty nullary constructor is required for deserializing the configuration. */
+    def this() = this(null)
+
+    override def getVersion = TraversableSerializerConfigSnapshot.VERSION
+  }
+
+  object TraversableSerializerConfigSnapshot {
+    val VERSION = 1
   }
 }


[12/12] flink git commit: [FLINK-6830] [fileSource] Port continuous file reader migration tests for Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [fileSource] Port continuous file reader migration tests for Flink 1.3

This commit also consolidates all Flink 1.1 and 1.2 migration tests into
a single ContinuousFileProcessingMigrationTest class. Parameterization
is used to test restore from different previous Flink versions.

This closes #4059.


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

Branch: refs/heads/release-1.3
Commit: d4a646a035366918a100f64428c471464870b8d0
Parents: e5a435b
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sun Jun 4 01:42:04 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 19:11:43 2017 +0200

----------------------------------------------------------------------
 ...inuousFileProcessingFrom11MigrationTest.java | 402 ------------------
 ...inuousFileProcessingFrom12MigrationTest.java | 366 ----------------
 .../ContinuousFileProcessingMigrationTest.java  | 423 +++++++++++++++++++
 ...gration-test-1496532000000-flink1.3-snapshot | Bin 0 -> 537 bytes
 .../reader-migration-test-flink1.3-snapshot     | Bin 0 -> 2823 bytes
 5 files changed, 423 insertions(+), 768 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d4a646a0/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom11MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom11MigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom11MigrationTest.java
deleted file mode 100644
index ec5e1ad..0000000
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom11MigrationTest.java
+++ /dev/null
@@ -1,402 +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.hdfstests;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
-import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-
-public class ContinuousFileProcessingFrom11MigrationTest {
-
-	private static final int NO_OF_FILES = 5;
-	private static final int LINES_PER_FILE = 10;
-
-	private static final long INTERVAL = 100;
-
-	private static File baseDir;
-
-	private static FileSystem hdfs;
-	private static String hdfsURI;
-	private static MiniDFSCluster hdfsCluster;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	//						PREPARING FOR THE TESTS
-
-	@BeforeClass
-	public static void createHDFS() {
-		try {
-			baseDir = tempFolder.newFolder().getAbsoluteFile();
-			FileUtil.fullyDelete(baseDir);
-
-			Configuration hdConf = new Configuration();
-			hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
-			hdConf.set("dfs.block.size", String.valueOf(1048576)); // this is the minimum we can set.
-
-			MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
-			hdfsCluster = builder.build();
-
-			hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":" + hdfsCluster.getNameNodePort() +"/";
-			hdfs = new org.apache.hadoop.fs.Path(hdfsURI).getFileSystem(hdConf);
-
-		} catch(Throwable e) {
-			e.printStackTrace();
-			Assert.fail("Test failed " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		try {
-			FileUtil.fullyDelete(baseDir);
-			hdfsCluster.shutdown();
-		} catch (Throwable t) {
-			throw new RuntimeException(t);
-		}
-	}
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = ContinuousFileProcessingFrom11MigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		return resource.getFile();
-	}
-
-	//						END OF PREPARATIONS
-
-	//						TESTS
-
-	@Test
-	public void testReaderSnapshotRestore() throws Exception {
-
-		/*
-
-		FileInputSplit split1 =
-			new FileInputSplit(3, new Path("test/test1"), 0, 100, null);
-		FileInputSplit split2 =
-			new FileInputSplit(2, new Path("test/test2"), 101, 200, null);
-		FileInputSplit split3 =
-			new FileInputSplit(1, new Path("test/test2"), 0, 100, null);
-		FileInputSplit split4 =
-			new FileInputSplit(0, new Path("test/test3"), 0, 100, null);
-
-		final OneShotLatch latch = new OneShotLatch();
-		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(hdfsURI));
-		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
-		ContinuousFileReaderOperator<FileInputSplit, ?> initReader = new ContinuousFileReaderOperator<>(format);
-		initReader.setOutputType(typeInfo, new ExecutionConfig());
-		OneInputStreamOperatorTestHarness<FileInputSplit, FileInputSplit> initTestInstance =
-			new OneInputStreamOperatorTestHarness<>(initReader);
-		initTestInstance.setTimeCharacteristic(TimeCharacteristic.EventTime);
-		initTestInstance.open();
-		// create some state in the reader
-		initTestInstance.processElement(new StreamRecord<>(split1));
-		initTestInstance.processElement(new StreamRecord<>(split2));
-		initTestInstance.processElement(new StreamRecord<>(split3));
-		initTestInstance.processElement(new StreamRecord<>(split4));
-		// take a snapshot of the operator's state. This will be used
-		// to initialize another reader and compare the results of the
-		// two operators.
-		final StreamTaskState snapshot;
-		synchronized (initTestInstance.getCheckpointLock()) {
-			snapshot = initTestInstance.snapshot(0L, 0L);
-		}
-
-		initTestInstance.snaphotToFile(snapshot, "src/test/resources/reader-migration-test-flink1.1-snapshot");
-
-		*/
-		TimestampedFileInputSplit split1 =
-			new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
-
-		TimestampedFileInputSplit split2 =
-			new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
-
-		TimestampedFileInputSplit split3 =
-			new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
-
-		TimestampedFileInputSplit split4 =
-			new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
-
-
-		final OneShotLatch latch = new OneShotLatch();
-
-		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(hdfsURI));
-		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
-
-		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
-		initReader.setOutputType(typeInfo, new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> initTestInstance =
-			new OneInputStreamOperatorTestHarness<>(initReader);
-		initTestInstance.setTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		initTestInstance.setup();
-		initTestInstance.initializeStateFromLegacyCheckpoint(getResourceFilename("reader-migration-test-flink1.1-snapshot"));
-		initTestInstance.open();
-
-		latch.trigger();
-
-		// ... and wait for the operators to close gracefully
-
-		synchronized (initTestInstance.getCheckpointLock()) {
-			initTestInstance.close();
-		}
-
-		FileInputSplit fsSplit1 = createSplitFromTimestampedSplit(split1);
-		FileInputSplit fsSplit2 = createSplitFromTimestampedSplit(split2);
-		FileInputSplit fsSplit3 = createSplitFromTimestampedSplit(split3);
-		FileInputSplit fsSplit4 = createSplitFromTimestampedSplit(split4);
-
-		// compare if the results contain what they should contain and also if
-		// they are the same, as they should.
-
-		Assert.assertTrue(initTestInstance.getOutput().contains(new StreamRecord<>(fsSplit1)));
-		Assert.assertTrue(initTestInstance.getOutput().contains(new StreamRecord<>(fsSplit2)));
-		Assert.assertTrue(initTestInstance.getOutput().contains(new StreamRecord<>(fsSplit3)));
-		Assert.assertTrue(initTestInstance.getOutput().contains(new StreamRecord<>(fsSplit4)));
-	}
-
-	private FileInputSplit createSplitFromTimestampedSplit(TimestampedFileInputSplit split) {
-		Preconditions.checkNotNull(split);
-
-		return new FileInputSplit(
-			split.getSplitNumber(),
-			split.getPath(),
-			split.getStart(),
-			split.getLength(),
-			split.getHostnames()
-		);
-	}
-
-	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
-
-		private static final long serialVersionUID = -6727603565381560267L;
-
-		private final OneShotLatch latch;
-
-		private FileInputSplit split;
-
-		private boolean reachedEnd;
-
-		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
-			super(filePath);
-			this.latch = latch;
-			this.reachedEnd = false;
-		}
-
-		@Override
-		public void open(FileInputSplit fileSplit) throws IOException {
-			this.split = fileSplit;
-			this.reachedEnd = false;
-		}
-
-		@Override
-		public boolean reachedEnd() throws IOException {
-			if (!latch.isTriggered()) {
-				try {
-					latch.await();
-				} catch (InterruptedException e) {
-					e.printStackTrace();
-				}
-			}
-			return reachedEnd;
-		}
-
-		@Override
-		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
-			this.reachedEnd = true;
-			return split;
-		}
-
-		@Override
-		public void close() {
-
-		}
-	}
-
-	////				Monitoring Function Tests				//////
-
-	@Test
-	public void testFunctionRestore() throws Exception {
-
-		/*
-		org.apache.hadoop.fs.Path path = null;
-		long fileModTime = Long.MIN_VALUE;
-		for (int i = 0; i < 1; i++) {
-			Tuple2<org.apache.hadoop.fs.Path, String> file = fillWithData(hdfsURI, "file", i, "This is test line.");
-			path = file.f0;
-			fileModTime = hdfs.getFileStatus(file.f0).getModificationTime();
-		}
-
-		TextInputFormat format = new TextInputFormat(new Path(hdfsURI));
-
-		final ContinuousFileMonitoringFunction<String> monitoringFunction =
-			new ContinuousFileMonitoringFunction<>(format, format.getFilePath().toString(), new PathFilter(), FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
-
-		StreamSource<FileInputSplit, ContinuousFileMonitoringFunction<String>> src =
-			new StreamSource<>(monitoringFunction);
-
-		final OneInputStreamOperatorTestHarness<Void, FileInputSplit> testHarness =
-			new OneInputStreamOperatorTestHarness<>(src);
-		testHarness.open();
-
-		final Throwable[] error = new Throwable[1];
-
-		final OneShotLatch latch = new OneShotLatch();
-
-		// run the source asynchronously
-		Thread runner = new Thread() {
-			@Override
-			public void run() {
-				try {
-					monitoringFunction.run(new DummySourceContext() {
-						@Override
-						public void collect(FileInputSplit element) {
-							latch.trigger();
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		StreamTaskState snapshot = testHarness.snapshot(0, 0);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/monitoring-function-migration-test-" + fileModTime +"-flink1.1-snapshot");
-		monitoringFunction.cancel();
-		runner.join();
-
-		testHarness.close();
-		*/
-
-		Long expectedModTime = Long.parseLong("1482144479339");
-		TextInputFormat format = new TextInputFormat(new Path(hdfsURI));
-
-		final ContinuousFileMonitoringFunction<String> monitoringFunction =
-			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
-
-		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
-			new StreamSource<>(monitoringFunction);
-
-		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
-			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
-		testHarness.setup();
-		testHarness.initializeStateFromLegacyCheckpoint(getResourceFilename("monitoring-function-migration-test-1482144479339-flink1.1-snapshot"));
-		testHarness.open();
-
-		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
-
-	}
-
-	///////////				Source Contexts Used by the tests				/////////////////
-
-	private static abstract class DummySourceContext
-		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
-
-		private final Object lock = new Object();
-
-		@Override
-		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lock;
-		}
-
-		@Override
-		public void close() {
-		}
-	}
-
-	/////////				Auxiliary Methods				/////////////
-
-	/**
-	 * Create a file with pre-determined String format of the form:
-	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
-	 * */
-	private Tuple2<org.apache.hadoop.fs.Path, String> createFileAndFillWithData(
-		String base, String fileName, int fileIdx, String sampleLine) throws IOException {
-
-		assert (hdfs != null);
-
-		org.apache.hadoop.fs.Path file = new org.apache.hadoop.fs.Path(base + "/" + fileName + fileIdx);
-		Assert.assertFalse(hdfs.exists(file));
-
-		org.apache.hadoop.fs.Path tmp = new org.apache.hadoop.fs.Path(base + "/." + fileName + fileIdx);
-		FSDataOutputStream stream = hdfs.create(tmp);
-		StringBuilder str = new StringBuilder();
-		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));
-		}
-		stream.close();
-
-		hdfs.rename(tmp, file);
-
-		Assert.assertTrue("No result file present", hdfs.exists(file));
-		return new Tuple2<>(file, str.toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/d4a646a0/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java
deleted file mode 100644
index bf09447..0000000
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java
+++ /dev/null
@@ -1,366 +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.hdfstests;
-
-import java.io.FileOutputStream;
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
-import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.flink.util.Preconditions;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-
-public class ContinuousFileProcessingFrom12MigrationTest {
-
-	private static final int LINES_PER_FILE = 10;
-
-	private static final long INTERVAL = 100;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	/**
-	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
-	 */
-	@Ignore
-	@Test
-	public void writeReaderSnapshot() throws Exception {
-
-		File testFolder = tempFolder.newFolder();
-
-		TimestampedFileInputSplit split1 =
-				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
-
-		TimestampedFileInputSplit split2 =
-				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
-
-		TimestampedFileInputSplit split3 =
-				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
-
-		TimestampedFileInputSplit split4 =
-				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
-
-		// this always blocks to ensure that the reader doesn't to any actual processing so that
-		// we keep the state for the four splits
-		final OneShotLatch blockingLatch = new OneShotLatch();
-		BlockingFileInputFormat format = new BlockingFileInputFormat(blockingLatch, new Path(testFolder.getAbsolutePath()));
-
-		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
-		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
-				format);
-		initReader.setOutputType(typeInfo, new ExecutionConfig());
-		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
-				new OneInputStreamOperatorTestHarness<>(initReader);
-		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
-		testHarness.open();
-		// create some state in the reader
-		testHarness.processElement(new StreamRecord<>(split1));
-		testHarness.processElement(new StreamRecord<>(split2));
-		testHarness.processElement(new StreamRecord<>(split3));
-		testHarness.processElement(new StreamRecord<>(split4));
-		// take a snapshot of the operator's state. This will be used
-		// to initialize another reader and compare the results of the
-		// two operators.
-
-		final OperatorStateHandles snapshot;
-		synchronized (testHarness.getCheckpointLock()) {
-			snapshot = testHarness.snapshot(0L, 0L);
-		}
-
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
-	}
-
-	@Test
-	public void testReaderRestore() throws Exception {
-		File testFolder = tempFolder.newFolder();
-
-		final OneShotLatch latch = new OneShotLatch();
-
-		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
-		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
-
-		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
-		initReader.setOutputType(typeInfo, new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
-			new OneInputStreamOperatorTestHarness<>(initReader);
-		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		testHarness.setup();
-		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
-				OperatorSnapshotUtil.getResourceFilename(
-						"reader-migration-test-flink1.2-snapshot"));
-		testHarness.initializeState(operatorStateHandles);
-		testHarness.open();
-
-		latch.trigger();
-
-		// ... and wait for the operators to close gracefully
-
-		synchronized (testHarness.getCheckpointLock()) {
-			testHarness.close();
-		}
-
-		TimestampedFileInputSplit split1 =
-				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
-
-		TimestampedFileInputSplit split2 =
-				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
-
-		TimestampedFileInputSplit split3 =
-				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
-
-		TimestampedFileInputSplit split4 =
-				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
-
-		// compare if the results contain what they should contain and also if
-		// they are the same, as they should.
-
-		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
-		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
-		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
-		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
-	 */
-	@Ignore
-	@Test
-	public void writeMonitoringSourceSnapshot() throws Exception {
-
-		File testFolder = tempFolder.newFolder();
-
-		long fileModTime = Long.MIN_VALUE;
-		for (int i = 0; i < 1; i++) {
-			Tuple2<File, String> file = createFileAndFillWithData(testFolder, "file", i, "This is test line.");
-			fileModTime = file.f0.lastModified();
-		}
-
-		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
-
-		final ContinuousFileMonitoringFunction<String> monitoringFunction =
-			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
-
-		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
-			new StreamSource<>(monitoringFunction);
-
-		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
-				new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
-
-		testHarness.open();
-
-		final Throwable[] error = new Throwable[1];
-
-		final OneShotLatch latch = new OneShotLatch();
-
-		// run the source asynchronously
-		Thread runner = new Thread() {
-			@Override
-			public void run() {
-				try {
-					monitoringFunction.run(new DummySourceContext() {
-						@Override
-						public void collect(TimestampedFileInputSplit element) {
-							latch.trigger();
-						}
-
-						@Override
-						public void markAsTemporarilyIdle() {
-
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		final OperatorStateHandles snapshot;
-		synchronized (testHarness.getCheckpointLock()) {
-			snapshot = testHarness.snapshot(0L, 0L);
-		}
-
-		OperatorSnapshotUtil.writeStateHandle(
-				snapshot,
-				"src/test/resources/monitoring-function-migration-test-" + fileModTime +"-flink1.2-snapshot");
-
-		monitoringFunction.cancel();
-		runner.join();
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testMonitoringSourceRestore() throws Exception {
-
-		File testFolder = tempFolder.newFolder();
-
-		Long expectedModTime = Long.parseLong("1493116191000");
-		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
-
-		final ContinuousFileMonitoringFunction<String> monitoringFunction =
-			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
-
-		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
-			new StreamSource<>(monitoringFunction);
-
-		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
-			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
-
-		testHarness.setup();
-		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
-				OperatorSnapshotUtil.getResourceFilename(
-						"monitoring-function-migration-test-1493116191000-flink1.2-snapshot"));
-
-		testHarness.initializeState(operatorStateHandles);
-		testHarness.open();
-
-		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
-
-	}
-
-	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
-
-		private static final long serialVersionUID = -6727603565381560267L;
-
-		private final OneShotLatch latch;
-
-		private FileInputSplit split;
-
-		private boolean reachedEnd;
-
-		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
-			super(filePath);
-			this.latch = latch;
-			this.reachedEnd = false;
-		}
-
-		@Override
-		public void open(FileInputSplit fileSplit) throws IOException {
-			this.split = fileSplit;
-			this.reachedEnd = false;
-		}
-
-		@Override
-		public boolean reachedEnd() throws IOException {
-			if (!latch.isTriggered()) {
-				try {
-					latch.await();
-				} catch (InterruptedException e) {
-					e.printStackTrace();
-				}
-			}
-			return reachedEnd;
-		}
-
-		@Override
-		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
-			this.reachedEnd = true;
-			return split;
-		}
-
-		@Override
-		public void close() {
-
-		}
-	}
-
-	private static abstract class DummySourceContext
-		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
-
-		private final Object lock = new Object();
-
-		@Override
-		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lock;
-		}
-
-		@Override
-		public void close() {
-		}
-	}
-
-	/**
-	 * Create a file with pre-determined String format of the form:
-	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
-	 * */
-	private Tuple2<File, String> createFileAndFillWithData(
-		File base, String fileName, int fileIdx, String sampleLine) throws IOException {
-
-		File file = new File(base, fileName + fileIdx);
-		Assert.assertFalse(file.exists());
-
-		File tmp = new File(base, "." + fileName + fileIdx);
-		FileOutputStream stream = new FileOutputStream(tmp);
-		StringBuilder str = new StringBuilder();
-		for (int i = 0; i < LINES_PER_FILE; i++) {
-			String line = fileIdx +": "+ sampleLine + " " + i +"\n";
-			str.append(line);
-			stream.write(line.getBytes());
-		}
-		stream.close();
-
-		FileUtils.moveFile(tmp, file);
-
-		Assert.assertTrue("No result file present", file.exists());
-		return new Tuple2<>(file, str.toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/d4a646a0/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
new file mode 100644
index 0000000..89776eb
--- /dev/null
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
@@ -0,0 +1,423 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.hdfstests;
+
+import java.io.FileOutputStream;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.io.FileInputFormat;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.io.TextInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
+import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
+import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Tests that verify the migration from previous Flink version snapshots.
+ */
+@RunWith(Parameterized.class)
+public class ContinuousFileProcessingMigrationTest {
+
+	private static final int LINES_PER_FILE = 10;
+
+	private static final long INTERVAL = 100;
+
+	@Parameterized.Parameters(name = "Migration Savepoint / Mod Time: {0}")
+	public static Collection<Tuple2<MigrationVersion, Long>> parameters () {
+		return Arrays.asList(
+			Tuple2.of(MigrationVersion.v1_1, 1482144479339L),
+			Tuple2.of(MigrationVersion.v1_2, 1493116191000L),
+			Tuple2.of(MigrationVersion.v1_3, 1496532000000L));
+	}
+
+	/**
+	 * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+	 * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints
+	 */
+	private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+	private final MigrationVersion testMigrateVersion;
+	private final Long expectedModTime;
+
+	public ContinuousFileProcessingMigrationTest(Tuple2<MigrationVersion, Long> migrationVersionAndModTime) {
+		this.testMigrateVersion = migrationVersionAndModTime.f0;
+		this.expectedModTime = migrationVersionAndModTime.f1;
+	}
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	/**
+	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
+	 */
+	@Ignore
+	@Test
+	public void writeReaderSnapshot() throws Exception {
+
+		File testFolder = tempFolder.newFolder();
+
+		TimestampedFileInputSplit split1 =
+				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
+
+		TimestampedFileInputSplit split2 =
+				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
+
+		TimestampedFileInputSplit split3 =
+				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
+
+		TimestampedFileInputSplit split4 =
+				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
+
+		// this always blocks to ensure that the reader doesn't to any actual processing so that
+		// we keep the state for the four splits
+		final OneShotLatch blockingLatch = new OneShotLatch();
+		BlockingFileInputFormat format = new BlockingFileInputFormat(blockingLatch, new Path(testFolder.getAbsolutePath()));
+
+		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
+		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
+				format);
+		initReader.setOutputType(typeInfo, new ExecutionConfig());
+		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
+				new OneInputStreamOperatorTestHarness<>(initReader);
+		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
+		testHarness.open();
+		// create some state in the reader
+		testHarness.processElement(new StreamRecord<>(split1));
+		testHarness.processElement(new StreamRecord<>(split2));
+		testHarness.processElement(new StreamRecord<>(split3));
+		testHarness.processElement(new StreamRecord<>(split4));
+		// take a snapshot of the operator's state. This will be used
+		// to initialize another reader and compare the results of the
+		// two operators.
+
+		final OperatorStateHandles snapshot;
+		synchronized (testHarness.getCheckpointLock()) {
+			snapshot = testHarness.snapshot(0L, 0L);
+		}
+
+		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink" + flinkGenerateSavepointVersion + "-snapshot");
+	}
+
+	@Test
+	public void testReaderRestore() throws Exception {
+		File testFolder = tempFolder.newFolder();
+
+		final OneShotLatch latch = new OneShotLatch();
+
+		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
+		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
+
+		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
+		initReader.setOutputType(typeInfo, new ExecutionConfig());
+
+		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
+			new OneInputStreamOperatorTestHarness<>(initReader);
+		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"reader-migration-test-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		latch.trigger();
+
+		// ... and wait for the operators to close gracefully
+
+		synchronized (testHarness.getCheckpointLock()) {
+			testHarness.close();
+		}
+
+		TimestampedFileInputSplit split1 =
+				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
+
+		TimestampedFileInputSplit split2 =
+				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
+
+		TimestampedFileInputSplit split3 =
+				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
+
+		TimestampedFileInputSplit split4 =
+				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
+
+		// compare if the results contain what they should contain and also if
+		// they are the same, as they should.
+
+		if (testMigrateVersion == MigrationVersion.v1_1) {
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(createSplitFromTimestampedSplit(split1))));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(createSplitFromTimestampedSplit(split2))));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(createSplitFromTimestampedSplit(split3))));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(createSplitFromTimestampedSplit(split4))));
+		} else {
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
+			Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
+		}
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
+	 */
+	@Ignore
+	@Test
+	public void writeMonitoringSourceSnapshot() throws Exception {
+
+		File testFolder = tempFolder.newFolder();
+
+		long fileModTime = Long.MIN_VALUE;
+		for (int i = 0; i < 1; i++) {
+			Tuple2<File, String> file = createFileAndFillWithData(testFolder, "file", i, "This is test line.");
+			fileModTime = file.f0.lastModified();
+		}
+
+		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
+
+		final ContinuousFileMonitoringFunction<String> monitoringFunction =
+			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
+
+		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
+			new StreamSource<>(monitoringFunction);
+
+		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
+				new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
+
+		testHarness.open();
+
+		final Throwable[] error = new Throwable[1];
+
+		final OneShotLatch latch = new OneShotLatch();
+
+		// run the source asynchronously
+		Thread runner = new Thread() {
+			@Override
+			public void run() {
+				try {
+					monitoringFunction.run(new DummySourceContext() {
+						@Override
+						public void collect(TimestampedFileInputSplit element) {
+							latch.trigger();
+						}
+
+						@Override
+						public void markAsTemporarilyIdle() {
+
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		final OperatorStateHandles snapshot;
+		synchronized (testHarness.getCheckpointLock()) {
+			snapshot = testHarness.snapshot(0L, 0L);
+		}
+
+		OperatorSnapshotUtil.writeStateHandle(
+				snapshot,
+				"src/test/resources/monitoring-function-migration-test-" + fileModTime + "-flink" + flinkGenerateSavepointVersion + "-snapshot");
+
+		monitoringFunction.cancel();
+		runner.join();
+
+		testHarness.close();
+	}
+
+	@Test
+	public void testMonitoringSourceRestore() throws Exception {
+
+		File testFolder = tempFolder.newFolder();
+
+		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
+
+		final ContinuousFileMonitoringFunction<String> monitoringFunction =
+			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
+
+		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
+			new StreamSource<>(monitoringFunction);
+
+		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
+			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
+
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"monitoring-function-migration-test-" + expectedModTime + "-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
+
+	}
+
+	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
+
+		private static final long serialVersionUID = -6727603565381560267L;
+
+		private final OneShotLatch latch;
+
+		private FileInputSplit split;
+
+		private boolean reachedEnd;
+
+		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
+			super(filePath);
+			this.latch = latch;
+			this.reachedEnd = false;
+		}
+
+		@Override
+		public void open(FileInputSplit fileSplit) throws IOException {
+			this.split = fileSplit;
+			this.reachedEnd = false;
+		}
+
+		@Override
+		public boolean reachedEnd() throws IOException {
+			if (!latch.isTriggered()) {
+				try {
+					latch.await();
+				} catch (InterruptedException e) {
+					e.printStackTrace();
+				}
+			}
+			return reachedEnd;
+		}
+
+		@Override
+		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
+			this.reachedEnd = true;
+			return split;
+		}
+
+		@Override
+		public void close() {
+
+		}
+	}
+
+	private static abstract class DummySourceContext
+		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
+
+		private final Object lock = new Object();
+
+		@Override
+		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+		}
+	}
+
+	/**
+	 * Create a file with pre-determined String format of the form:
+	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
+	 * */
+	private Tuple2<File, String> createFileAndFillWithData(
+		File base, String fileName, int fileIdx, String sampleLine) throws IOException {
+
+		File file = new File(base, fileName + fileIdx);
+		Assert.assertFalse(file.exists());
+
+		File tmp = new File(base, "." + fileName + fileIdx);
+		FileOutputStream stream = new FileOutputStream(tmp);
+		StringBuilder str = new StringBuilder();
+		for (int i = 0; i < LINES_PER_FILE; i++) {
+			String line = fileIdx +": "+ sampleLine + " " + i +"\n";
+			str.append(line);
+			stream.write(line.getBytes());
+		}
+		stream.close();
+
+		FileUtils.moveFile(tmp, file);
+
+		Assert.assertTrue("No result file present", file.exists());
+		return new Tuple2<>(file, str.toString());
+	}
+
+	private FileInputSplit createSplitFromTimestampedSplit(TimestampedFileInputSplit split) {
+		checkNotNull(split);
+
+		return new FileInputSplit(
+			split.getSplitNumber(),
+			split.getPath(),
+			split.getStart(),
+			split.getLength(),
+			split.getHostnames()
+		);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d4a646a0/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1496532000000-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1496532000000-flink1.3-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1496532000000-flink1.3-snapshot
new file mode 100644
index 0000000..7ed677b
Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1496532000000-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/d4a646a0/flink-fs-tests/src/test/resources/reader-migration-test-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/resources/reader-migration-test-flink1.3-snapshot b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.3-snapshot
new file mode 100644
index 0000000..bb612bd
Binary files /dev/null and b/flink-fs-tests/src/test/resources/reader-migration-test-flink1.3-snapshot differ


[06/12] flink git commit: [FLINK-6830] [tests] Port topology change migration ITCases for Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [tests] Port topology change migration ITCases for Flink 1.3


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

Branch: refs/heads/release-1.3
Commit: 8f72070619aef229f348a3d85d91094c8a640fbf
Parents: 7a4d016
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sat Jun 3 21:56:54 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:52:23 2017 +0200

----------------------------------------------------------------------
 .../AbstractOperatorRestoreTestBase.java        |  14 ++--
 .../state/operator/restore/ExecutionMode.java   |   6 +-
 .../AbstractKeyedOperatorRestoreTestBase.java   |  70 +++++++++++++++++++
 .../restore/keyed/KeyedComplexChainTest.java    |  22 +-----
 .../state/operator/restore/keyed/KeyedJob.java  |   7 +-
 ...AbstractNonKeyedOperatorRestoreTestBase.java |  26 ++++++-
 .../restore/unkeyed/ChainBreakTest.java         |   4 ++
 .../unkeyed/ChainLengthDecreaseTest.java        |   4 ++
 .../unkeyed/ChainLengthIncreaseTest.java        |   4 ++
 .../restore/unkeyed/ChainOrderTest.java         |   4 ++
 .../restore/unkeyed/ChainUnionTest.java         |   4 ++
 .../operator/restore/unkeyed/NonKeyedJob.java   |   6 +-
 .../complexKeyed-flink1.2/_metadata             | Bin 0 -> 137490 bytes
 .../complexKeyed-flink1.3/_metadata             | Bin 0 -> 163526 bytes
 .../operatorstate/complexKeyed/_metadata        | Bin 137490 -> 0 bytes
 .../operatorstate/nonKeyed-flink1.2/_metadata   | Bin 0 -> 3212 bytes
 .../operatorstate/nonKeyed-flink1.3/_metadata   | Bin 0 -> 6248 bytes
 .../resources/operatorstate/nonKeyed/_metadata  | Bin 3212 -> 0 bytes
 18 files changed, 130 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 82e8d94..f087cf4 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
@@ -65,11 +65,11 @@ import java.net.URL;
 import java.util.concurrent.TimeUnit;
 
 /**
- * Abstract class to verify that it is possible to migrate a 1.2 savepoint to 1.3 and that the topology can be modified
- * from that point on.
+ * 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:
- * Step 1: Migrate the job to 1.3 by submitting the same job used for the 1.2 savepoint, and create a new savepoint.
+ * 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.
  */
 public abstract class AbstractOperatorRestoreTestBase extends TestLogger {
@@ -160,9 +160,9 @@ public abstract class AbstractOperatorRestoreTestBase extends TestLogger {
 
 	@Test
 	public void testMigrationAndRestore() throws Throwable {
-		// submit 1.2 job and create a migrated 1.3 savepoint
+		// submit job with old version savepoint and create a migrated savepoint in the new version
 		String savepointPath = migrateJob();
-		// restore from migrated 1.3 savepoint
+		// restore from migrated new version savepoint
 		restoreJob(savepointPath);
 	}
 
@@ -256,14 +256,14 @@ public abstract class AbstractOperatorRestoreTestBase extends TestLogger {
 	}
 
 	/**
-	 * Recreates the job used to create the 1.2 savepoint.
+	 * Recreates the job used to create the new version savepoint.
 	 *
 	 * @param env StreamExecutionEnvironment to use
 	 */
 	protected abstract void createMigrationJob(StreamExecutionEnvironment env);
 
 	/**
-	 * Creates a modified version of the job used to create the 1.2 savepoint.
+	 * Creates a modified version of the job used to create the new version savepoint.
 	 *
 	 * @param env StreamExecutionEnvironment to use
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 f333aca..ae9fb21 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
@@ -20,9 +20,9 @@ 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 1.2 savepoint.
- * {@link ExecutionMode#MIGRATE} should be used when migrating the 1.2 savepoint to 1.3.
- * {@link ExecutionMode#RESTORE} should be used when restoring from the migrated 1.3 savepoint.
+ * {@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.
  */
 public enum ExecutionMode {
 	GENERATE,

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.java
new file mode 100644
index 0000000..1b66c21
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/AbstractKeyedOperatorRestoreTestBase.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.state.operator.restore.keyed;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase;
+import org.apache.flink.test.state.operator.restore.ExecutionMode;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Base class for all keyed operator restore tests.
+ */
+@RunWith(Parameterized.class)
+public abstract class AbstractKeyedOperatorRestoreTestBase extends AbstractOperatorRestoreTestBase {
+
+	private final String savepointPath;
+
+	@Parameterized.Parameters(name = "Migrate Savepoint: {0}")
+	public static Collection<String> parameters () {
+		return Arrays.asList(
+			"complexKeyed-flink1.2",
+			"complexKeyed-flink1.3");
+	}
+
+	public AbstractKeyedOperatorRestoreTestBase(String savepointPath) {
+		this.savepointPath = savepointPath;
+	}
+
+	@Override
+	public void createMigrationJob(StreamExecutionEnvironment env) {
+		/**
+		 * Source -> keyBy -> C(Window -> StatefulMap1 -> StatefulMap2)
+		 */
+		SingleOutputStreamOperator<Tuple2<Integer, Integer>> source = KeyedJob.createIntegerTupleSource(env, ExecutionMode.MIGRATE);
+
+		SingleOutputStreamOperator<Integer> window = KeyedJob.createWindowFunction(ExecutionMode.MIGRATE, source);
+
+		SingleOutputStreamOperator<Integer> first = KeyedJob.createFirstStatefulMap(ExecutionMode.MIGRATE, window);
+
+		SingleOutputStreamOperator<Integer> second = KeyedJob.createSecondStatefulMap(ExecutionMode.MIGRATE, first);
+	}
+
+	@Override
+	protected String getMigrationSavepointName() {
+		return savepointPath;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 28cd15a..605722d 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
@@ -20,23 +20,12 @@ package org.apache.flink.test.state.operator.restore.keyed;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase;
 import org.apache.flink.test.state.operator.restore.ExecutionMode;
 
-public class KeyedComplexChainTest extends AbstractOperatorRestoreTestBase {
+public class KeyedComplexChainTest extends AbstractKeyedOperatorRestoreTestBase {
 
-	@Override
-	public void createMigrationJob(StreamExecutionEnvironment env) {
-		/**
-		 * Source -> keyBy -> C(Window -> StatefulMap1 -> StatefulMap2)
-		 */
-		SingleOutputStreamOperator<Tuple2<Integer, Integer>> source = KeyedJob.createIntegerTupleSource(env, ExecutionMode.MIGRATE);
-
-		SingleOutputStreamOperator<Integer> window = KeyedJob.createWindowFunction(ExecutionMode.MIGRATE, source);
-
-		SingleOutputStreamOperator<Integer> first = KeyedJob.createFirstStatefulMap(ExecutionMode.MIGRATE, window);
-
-		SingleOutputStreamOperator<Integer> second = KeyedJob.createSecondStatefulMap(ExecutionMode.MIGRATE, first);
+	public KeyedComplexChainTest(String savepointPath) {
+		super(savepointPath);
 	}
 
 	@Override
@@ -53,9 +42,4 @@ public class KeyedComplexChainTest extends AbstractOperatorRestoreTestBase {
 		SingleOutputStreamOperator<Integer> first = KeyedJob.createFirstStatefulMap(ExecutionMode.RESTORE, second);
 		first.startNewChain();
 	}
-
-	@Override
-	protected final String getMigrationSavepointName() {
-		return "complexKeyed";
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 6add7b2..523e937 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
@@ -44,7 +44,8 @@ import java.util.Iterator;
 import java.util.List;
 
 /**
- * Savepoint generator to create the job used by the {@link KeyedComplexChainTest}.
+ * 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.
  */
@@ -236,8 +237,4 @@ public class KeyedJob {
 			}
 		}
 	}
-
-
-	private KeyedJob() {
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 5b51765..22fa7b2 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
@@ -23,6 +23,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.state.operator.restore.AbstractOperatorRestoreTestBase;
 import org.apache.flink.test.state.operator.restore.ExecutionMode;
 
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
 import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createFirstStatefulMap;
 import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createSecondStatefulMap;
 import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createSource;
@@ -30,10 +36,24 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
 import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.createThirdStatefulMap;
 
 /**
- * All classes extending this class will use the same savepoint and migration job.
+ * Base class for all non-keyed operator restore tests.
  */
+@RunWith(Parameterized.class)
 public abstract class AbstractNonKeyedOperatorRestoreTestBase extends AbstractOperatorRestoreTestBase {
 
+	private final String savepointPath;
+
+	@Parameterized.Parameters(name = "Migrate Savepoint: {0}")
+	public static Collection<String> parameters () {
+		return Arrays.asList(
+			"nonKeyed-flink1.2",
+			"nonKeyed-flink1.3");
+	}
+
+	public AbstractNonKeyedOperatorRestoreTestBase(String savepointPath) {
+		this.savepointPath = savepointPath;
+	}
+
 	@Override
 	public void createMigrationJob(StreamExecutionEnvironment env) {
 		/**
@@ -53,7 +73,7 @@ public abstract class AbstractNonKeyedOperatorRestoreTestBase extends AbstractOp
 	}
 
 	@Override
-	protected final String getMigrationSavepointName() {
-		return "nonKeyed";
+	protected String getMigrationSavepointName() {
+		return savepointPath;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 6838070..8055833 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
@@ -33,6 +33,10 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
  */
 public class ChainBreakTest extends AbstractNonKeyedOperatorRestoreTestBase {
 
+	public ChainBreakTest(String savepointPath) {
+		super(savepointPath);
+	}
+
 	@Override
 	public void createRestoredJob(StreamExecutionEnvironment env) {
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 e405e76..3235387 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
@@ -32,6 +32,10 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
  */
 public class ChainLengthDecreaseTest extends AbstractNonKeyedOperatorRestoreTestBase {
 
+	public ChainLengthDecreaseTest(String savepointPath) {
+		super(savepointPath);
+	}
+
 	@Override
 	public void createRestoredJob(StreamExecutionEnvironment env) {
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 b78aa10..a10f99c 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
@@ -33,6 +33,10 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
  */
 public class ChainLengthIncreaseTest extends AbstractNonKeyedOperatorRestoreTestBase {
 
+	public ChainLengthIncreaseTest(String savepointPath) {
+		super(savepointPath);
+	}
+
 	@Override
 	public void createRestoredJob(StreamExecutionEnvironment env) {
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 7c68b4e..0baa233 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
@@ -33,6 +33,10 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
  */
 public class ChainOrderTest extends AbstractNonKeyedOperatorRestoreTestBase {
 
+	public ChainOrderTest(String savepointPath) {
+		super(savepointPath);
+	}
+
 	@Override
 	public void createRestoredJob(StreamExecutionEnvironment env) {
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 3f2fba4..0d21e8a 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
@@ -33,6 +33,10 @@ import static org.apache.flink.test.state.operator.restore.unkeyed.NonKeyedJob.c
  */
 public class ChainUnionTest extends AbstractNonKeyedOperatorRestoreTestBase {
 
+	public ChainUnionTest(String savepointPath) {
+		super(savepointPath);
+	}
+
 	@Override
 	public void createRestoredJob(StreamExecutionEnvironment env) {
 		/**

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/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 32067b3..08a4c67 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
@@ -37,7 +37,8 @@ import java.util.Arrays;
 import java.util.List;
 
 /**
- * Savepoint generator to create the job used by the {@link AbstractNonKeyedOperatorRestoreTestBase}.
+ * 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.
  */
@@ -192,7 +193,4 @@ public class NonKeyedJob {
 			}
 		}
 	}
-
-	private NonKeyedJob() {
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.2/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.2/_metadata b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.2/_metadata
new file mode 100644
index 0000000..9e03876
Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.2/_metadata differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.3/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.3/_metadata b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.3/_metadata
new file mode 100644
index 0000000..a4f5a1e
Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/complexKeyed-flink1.3/_metadata differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/complexKeyed/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/complexKeyed/_metadata b/flink-tests/src/test/resources/operatorstate/complexKeyed/_metadata
deleted file mode 100644
index 9e03876..0000000
Binary files a/flink-tests/src/test/resources/operatorstate/complexKeyed/_metadata and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.2/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.2/_metadata b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.2/_metadata
new file mode 100644
index 0000000..8fcd1ea
Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.2/_metadata differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.3/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.3/_metadata b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.3/_metadata
new file mode 100644
index 0000000..46169e0
Binary files /dev/null and b/flink-tests/src/test/resources/operatorstate/nonKeyed-flink1.3/_metadata differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8f720706/flink-tests/src/test/resources/operatorstate/nonKeyed/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/operatorstate/nonKeyed/_metadata b/flink-tests/src/test/resources/operatorstate/nonKeyed/_metadata
deleted file mode 100644
index 8fcd1ea..0000000
Binary files a/flink-tests/src/test/resources/operatorstate/nonKeyed/_metadata and /dev/null differ


[09/12] flink git commit: [FLINK-6830] [kafka] Port migration tests for FlinkKafkaConsumerBase to Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [kafka] Port migration tests for FlinkKafkaConsumerBase to Flink 1.3

This commit also consolidates all tests for migration from Flink 1.1 and
1.2 for the FlinkKafkaConsumerBase to a single class
FlinkKafkaConsumerBaseMigrationTest. Parameterization is used to test
migration from different Flink version savepoints.


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

Branch: refs/heads/release-1.3
Commit: 74bb9a83a923a76443426e15ad4b4285bcfeb69d
Parents: 0544b44
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sun Jun 4 00:18:26 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 19:08:03 2017 +0200

----------------------------------------------------------------------
 ...inkKafkaConsumerBaseFrom11MigrationTest.java | 347 -----------------
 ...inkKafkaConsumerBaseFrom12MigrationTest.java | 338 -----------------
 .../FlinkKafkaConsumerBaseMigrationTest.java    | 374 +++++++++++++++++++
 ...migration-test-flink1.3-empty-state-snapshot | Bin 0 -> 473 bytes
 ...ka-consumer-migration-test-flink1.3-snapshot | Bin 0 -> 1255 bytes
 5 files changed, 374 insertions(+), 685 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/74bb9a83/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
deleted file mode 100644
index c07ebd5..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.SerializedValue;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
- * done using the Flink 1.1 {@link FlinkKafkaConsumerBase}.
- *
- * <p>For regenerating the binary snapshot file you have to run the commented out portion
- * of each test on a checkout of the Flink 1.1 branch.
- */
-public class FlinkKafkaConsumerBaseFrom11MigrationTest {
-
-	/** Test restoring from an legacy empty state, when no partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateNoPartitions() throws Exception {
-		final DummyFlinkKafkaConsumer<String> consumerFunction =
-			new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-empty-state-snapshot"));
-		testHarness.open();
-
-		// assert that no partitions were found and is empty
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// assert that no state was restored
-		Assert.assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/** Test restoring from an empty state taken using Flink 1.1, when some partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateWithPartitions() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-empty-state-snapshot"));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.1-empty-state-snapshot";
-		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
-		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
-		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("abc", 13), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("def", 7), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-
-		// assert that there are partitions and is identical to expected list
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		// assert that no state was restored
-		Assert.assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/** Test restoring from a non-empty state taken using Flink 1.1, when some partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot"));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot"
-		final HashMap<KafkaTopicPartition, Long> expectedState = new HashMap<>();
-		expectedState.put(new KafkaTopicPartition("abc", 13), 16768L);
-		expectedState.put(new KafkaTopicPartition("def", 7), 987654321L);
-
-		// assert that there are partitions and is identical to expected list
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
-		Assert.assertEquals(expectedState, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		// assert that state is correctly restored from legacy checkpoint
-		Assert.assertTrue(consumerFunction.getRestoredState() != null);
-		Assert.assertEquals(expectedState, consumerFunction.getRestoredState());
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = FlinkKafkaConsumerBaseFrom11MigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final List<KafkaTopicPartition> partitions;
-
-		@SuppressWarnings("unchecked")
-		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
-			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-			this.partitions = partitions;
-		}
-
-		@Override
-		protected AbstractFetcher<T, ?> createFetcher(
-				SourceContext<T> sourceContext,
-				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
-				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
-				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
-				StreamingRuntimeContext runtimeContext,
-				OffsetCommitMode offsetCommitMode) throws Exception {
-			return mock(AbstractFetcher.class);
-		}
-
-		@Override
-		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-			return partitions;
-		}
-
-		@Override
-		protected boolean getIsAutoCommitEnabled() {
-			return false;
-		}
-	}
-}
-
-/*
-	THE CODE FOR FLINK 1.1
-
-	@Test
-	public void testRestoreFromFlink11() throws Exception {
-		// --------------------------------------------------------------------
-		//   prepare fake states
-		// --------------------------------------------------------------------
-
-		final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
-		state1.put(new KafkaTopicPartition("abc", 13), 16768L);
-		state1.put(new KafkaTopicPartition("def", 7), 987654321L);
-
-		final OneShotLatch latch = new OneShotLatch();
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				latch.trigger();
-				return null;
-			}
-		}).when(fetcher).runFetchLoop();
-
-		when(fetcher.snapshotCurrentState()).thenReturn(state1);
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
-			new FetcherFactory<String>() {
-				private static final long serialVersionUID = -2803131905656983619L;
-
-				@Override
-				public AbstractFetcher<String, ?> createFetcher() {
-					return fetcher;
-				}
-			});
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final OneInputStreamOperatorTestHarness<Void, String> testHarness =
-			new OneInputStreamOperatorTestHarness<>(consumerOperator);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		testHarness.open();
-
-		final Throwable[] error = new Throwable[1];
-
-		// run the source asynchronously
-		Thread runner = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumerFunction.run(new DummySourceContext() {
-						@Override
-						public void collect(String element) {
-							latch.trigger();
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/kafka-consumer-migration-test-flink1.1-snapshot-2");
-		consumerOperator.run(new Object());
-
-		consumerOperator.close();
-		runner.join();
-
-		System.out.println("Killed");
-	}
-
-	private static abstract class DummySourceContext
-		implements SourceFunction.SourceContext<String> {
-
-		private final Object lock = new Object();
-
-		@Override
-		public void collectWithTimestamp(String element, long timestamp) {
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lock;
-		}
-
-		@Override
-		public void close() {
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
-
-	private interface FetcherFactory<T> extends Serializable {
-		AbstractFetcher<T, ?> createFetcher();
-	}
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final FetcherFactory<T> fetcherFactory;
-
-		@SuppressWarnings("unchecked")
-		public DummyFlinkKafkaConsumer(FetcherFactory<T> fetcherFactory) {
-			super((KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-
-			final List<KafkaTopicPartition> partitions = new ArrayList<>();
-			partitions.add(new KafkaTopicPartition("dummy-topic", 0));
-			setSubscribedPartitions(partitions);
-
-			this.fetcherFactory = fetcherFactory;
-		}
-
-		@Override
-		protected AbstractFetcher<T, ?> createFetcher(SourceContext<T> sourceContext, List<KafkaTopicPartition> thisSubtaskPartitions, SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, StreamingRuntimeContext runtimeContext) throws Exception {
-			return fetcherFactory.createFetcher();
-		}
-	}
-* */

http://git-wip-us.apache.org/repos/asf/flink/blob/74bb9a83/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
deleted file mode 100644
index f11bf9f..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
+++ /dev/null
@@ -1,338 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.SerializedValue;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
- * done using the Flink 1.2 {@link FlinkKafkaConsumerBase}.
- *
- * <p>For regenerating the binary snapshot files run {@link #writeSnapshot()} on the Flink 1.2
- * branch.
- */
-public class FlinkKafkaConsumerBaseFrom12MigrationTest {
-
-	final static HashMap<KafkaTopicPartition, Long> PARTITION_STATE = new HashMap<>();
-
-	static {
-		PARTITION_STATE.put(new KafkaTopicPartition("abc", 13), 16768L);
-		PARTITION_STATE.put(new KafkaTopicPartition("def", 7), 987654321L);
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeSnapshot() throws Exception {
-		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", PARTITION_STATE);
-
-		final HashMap<KafkaTopicPartition, Long> emptyState = new HashMap<>();
-		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-empty-state-snapshot", emptyState);
-	}
-
-	private void writeSnapshot(String path, HashMap<KafkaTopicPartition, Long> state) throws Exception {
-
-		final OneShotLatch latch = new OneShotLatch();
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				latch.trigger();
-				return null;
-			}
-		}).when(fetcher).runFetchLoop();
-
-		when(fetcher.snapshotCurrentState()).thenReturn(state);
-
-		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(fetcher, partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-				new StreamSource<>(consumerFunction);
-
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		testHarness.open();
-
-		final Throwable[] error = new Throwable[1];
-
-		// run the source asynchronously
-		Thread runner = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumerFunction.run(new DummySourceContext() {
-						@Override
-						public void collect(String element) {
-
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		final OperatorStateHandles snapshot;
-		synchronized (testHarness.getCheckpointLock()) {
-			snapshot = testHarness.snapshot(0L, 0L);
-		}
-
-		OperatorSnapshotUtil.writeStateHandle(snapshot, path);
-
-		consumerOperator.close();
-		runner.join();
-	}
-
-	/**
-	 * Test restoring from an legacy empty state, when no partitions could be found for topics.
-	 */
-	@Test
-	public void testRestoreFromEmptyStateNoPartitions() throws Exception {
-		final DummyFlinkKafkaConsumer<String> consumerFunction =
-				new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-empty-state-snapshot")));
-		testHarness.open();
-
-		// assert that no partitions were found and is empty
-		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// assert that no state was restored
-		assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/**
-	 * Test restoring from an empty state taken using Flink 1.2, when some partitions could be
-	 * found for topics.
-	 */
-	@Test
-	public void testRestoreFromEmptyStateWithPartitions() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-				new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-empty-state-snapshot")));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.2-empty-state-snapshot";
-		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
-		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
-		for (KafkaTopicPartition partition : PARTITION_STATE.keySet()) {
-			expectedSubscribedPartitionsWithStartOffsets.put(partition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-		}
-
-		// assert that there are partitions and is identical to expected list
-		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/**
-	 * Test restoring from a non-empty state taken using Flink 1.2, when some partitions could be
-	 * found for topics.
-	 */
-	@Test
-	public void testRestore() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-				new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
-		testHarness.open();
-
-		// assert that there are partitions and is identical to expected list
-		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
-		Assert.assertEquals(PARTITION_STATE, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		// assert that state is correctly restored from legacy checkpoint
-		assertTrue(consumerFunction.getRestoredState() != null);
-		Assert.assertEquals(PARTITION_STATE, consumerFunction.getRestoredState());
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final List<KafkaTopicPartition> partitions;
-
-		private final AbstractFetcher<T, ?> fetcher;
-
-		@SuppressWarnings("unchecked")
-		DummyFlinkKafkaConsumer(AbstractFetcher<T, ?> fetcher, List<KafkaTopicPartition> partitions) {
-			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-			this.fetcher = fetcher;
-			this.partitions = partitions;
-		}
-
-		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
-			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-			this.fetcher = mock(AbstractFetcher.class);
-			this.partitions = partitions;
-		}
-
-		@Override
-		protected AbstractFetcher<T, ?> createFetcher(
-				SourceContext<T> sourceContext,
-				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
-				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
-				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
-				StreamingRuntimeContext runtimeContext,
-				OffsetCommitMode offsetCommitMode) throws Exception {
-			return fetcher;
-		}
-
-		@Override
-		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-			return partitions;
-		}
-
-		@Override
-		protected boolean getIsAutoCommitEnabled() {
-			return false;
-		}
-	}
-
-
-	private static abstract class DummySourceContext
-			implements SourceFunction.SourceContext<String> {
-
-		private final Object lock = new Object();
-
-		@Override
-		public void collectWithTimestamp(String element, long timestamp) {
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lock;
-		}
-
-		@Override
-		public void close() {
-		}
-
-		@Override
-		public void markAsTemporarilyIdle() {
-
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/74bb9a83/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
new file mode 100644
index 0000000..70e60f3
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Collection;
+
+/**
+ * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
+ * done using previous Flink versions' {@link FlinkKafkaConsumerBase}.
+ *
+ * <p>For regenerating the binary snapshot files run {@link #writeSnapshot()} on the corresponding
+ * Flink release-* branch.
+ */
+@RunWith(Parameterized.class)
+public class FlinkKafkaConsumerBaseMigrationTest {
+
+	/**
+	 * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+	 * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints
+	 */
+	private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+	final static HashMap<KafkaTopicPartition, Long> PARTITION_STATE = new HashMap<>();
+
+	static {
+		PARTITION_STATE.put(new KafkaTopicPartition("abc", 13), 16768L);
+		PARTITION_STATE.put(new KafkaTopicPartition("def", 7), 987654321L);
+	}
+
+	private final MigrationVersion testMigrateVersion;
+
+	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
+	public static Collection<MigrationVersion> parameters () {
+		return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3);
+	}
+
+	public FlinkKafkaConsumerBaseMigrationTest(MigrationVersion testMigrateVersion) {
+		this.testMigrateVersion = testMigrateVersion;
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeSnapshot() throws Exception {
+		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-snapshot", PARTITION_STATE);
+
+		final HashMap<KafkaTopicPartition, Long> emptyState = new HashMap<>();
+		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-empty-state-snapshot", emptyState);
+	}
+
+	private void writeSnapshot(String path, HashMap<KafkaTopicPartition, Long> state) throws Exception {
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		when(fetcher.snapshotCurrentState()).thenReturn(state);
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(fetcher, partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		testHarness.open();
+
+		final Throwable[] error = new Throwable[1];
+
+		// run the source asynchronously
+		Thread runner = new Thread() {
+			@Override
+			public void run() {
+				try {
+					consumerFunction.run(new DummySourceContext() {
+						@Override
+						public void collect(String element) {
+
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		final OperatorStateHandles snapshot;
+		synchronized (testHarness.getCheckpointLock()) {
+			snapshot = testHarness.snapshot(0L, 0L);
+		}
+
+		OperatorSnapshotUtil.writeStateHandle(snapshot, path);
+
+		consumerOperator.close();
+		runner.join();
+	}
+
+	/**
+	 * Test restoring from an legacy empty state, when no partitions could be found for topics.
+	 */
+	@Test
+	public void testRestoreFromEmptyStateNoPartitions() throws Exception {
+		final DummyFlinkKafkaConsumer<String> consumerFunction =
+				new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+
+		// restore state from binary snapshot file
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"kafka-consumer-migration-test-flink" + testMigrateVersion + "-empty-state-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		// assert that no partitions were found and is empty
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// assert that no state was restored
+		assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/**
+	 * Test restoring from an empty state taken using a previous Flink version, when some partitions could be
+	 * found for topics.
+	 */
+	@Test
+	public void testRestoreFromEmptyStateWithPartitions() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+
+		// restore state from binary snapshot file
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"kafka-consumer-migration-test-flink" + testMigrateVersion + "-empty-state-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		// the expected state in "kafka-consumer-migration-test-flink*-empty-state-snapshot";
+		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
+		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
+		for (KafkaTopicPartition partition : PARTITION_STATE.keySet()) {
+			expectedSubscribedPartitionsWithStartOffsets.put(partition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
+		}
+
+		// assert that there are partitions and is identical to expected list
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/**
+	 * Test restoring from a non-empty state taken using a previous Flink version, when some partitions could be
+	 * found for topics.
+	 */
+	@Test
+	public void testRestore() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+
+		// restore state from binary snapshot file
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"kafka-consumer-migration-test-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		// assert that there are partitions and is identical to expected list
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
+		Assert.assertEquals(PARTITION_STATE, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		// assert that state is correctly restored from legacy checkpoint
+		assertTrue(consumerFunction.getRestoredState() != null);
+		Assert.assertEquals(PARTITION_STATE, consumerFunction.getRestoredState());
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		private final List<KafkaTopicPartition> partitions;
+
+		private final AbstractFetcher<T, ?> fetcher;
+
+		@SuppressWarnings("unchecked")
+		DummyFlinkKafkaConsumer(AbstractFetcher<T, ?> fetcher, List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcher = fetcher;
+			this.partitions = partitions;
+		}
+
+		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcher = mock(AbstractFetcher.class);
+			this.partitions = partitions;
+		}
+
+		@Override
+		protected AbstractFetcher<T, ?> createFetcher(
+				SourceContext<T> sourceContext,
+				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
+				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+				StreamingRuntimeContext runtimeContext,
+				OffsetCommitMode offsetCommitMode) throws Exception {
+			return fetcher;
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return partitions;
+		}
+
+		@Override
+		protected boolean getIsAutoCommitEnabled() {
+			return false;
+		}
+	}
+
+
+	private static abstract class DummySourceContext
+			implements SourceFunction.SourceContext<String> {
+
+		private final Object lock = new Object();
+
+		@Override
+		public void collectWithTimestamp(String element, long timestamp) {
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+		}
+
+		@Override
+		public void markAsTemporarilyIdle() {
+
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/74bb9a83/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-empty-state-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-empty-state-snapshot b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-empty-state-snapshot
new file mode 100644
index 0000000..1a5aad1
Binary files /dev/null and b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-empty-state-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/74bb9a83/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-snapshot b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-snapshot
new file mode 100644
index 0000000..dc820ef
Binary files /dev/null and b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.3-snapshot differ


[03/12] flink git commit: [FLINK-6853] [DataStream] Let StreamRecordSerializer be compatible with MultiplexingStreamRecordSerializer

Posted by tz...@apache.org.
[FLINK-6853] [DataStream] Let StreamRecordSerializer be compatible with MultiplexingStreamRecordSerializer

This commit lets StreamRecordSerializer.ensureCompatibility be tolerable
for config snapshots taken from the legacy
MultiplexingStreamRecordSerializer. This is required for users which
originally used MultiplexingStreamRecordSerializer to serialize stream
elements as part of their checkpointed state (e.g. FlinkCEP).

This closes #4079.


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

Branch: refs/heads/release-1.3
Commit: 1d89dd06c1f9b09420ad3ff095d0842b4a951938
Parents: e1e207c
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Tue Jun 6 10:41:03 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:52:00 2017 +0200

----------------------------------------------------------------------
 .../streamrecord/StreamElementSerializer.java   | 41 ++++++++++++--------
 1 file changed, 25 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1d89dd06/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
index 390ac9d..19a69f5 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
@@ -32,6 +32,7 @@ import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.migration.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
 
@@ -285,26 +286,34 @@ public final class StreamElementSerializer<T> extends TypeSerializer<StreamEleme
 
 	@Override
 	public CompatibilityResult<StreamElement> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig;
+
+		// we are compatible for data written by ourselves or the legacy MultiplexingStreamRecordSerializer
 		if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) {
-			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig =
+			previousTypeSerializerAndConfig =
 				((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-			CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousTypeSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousTypeSerializerAndConfig.f1,
-					typeSerializer);
-
-			if (!compatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (compatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new StreamElementSerializer<>(
-						new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-			}
+		} else if (configSnapshot instanceof MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) {
+			previousTypeSerializerAndConfig =
+				((MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
+		} else {
+			return CompatibilityResult.requiresMigration();
 		}
 
-		return CompatibilityResult.requiresMigration();
+		CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
+				previousTypeSerializerAndConfig.f0,
+				UnloadableDummyTypeSerializer.class,
+				previousTypeSerializerAndConfig.f1,
+				typeSerializer);
+
+		if (!compatResult.isRequiresMigration()) {
+			return CompatibilityResult.compatible();
+		} else if (compatResult.getConvertDeserializer() != null) {
+			return CompatibilityResult.requiresMigration(
+				new StreamElementSerializer<>(
+					new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
+		} else {
+			return CompatibilityResult.requiresMigration();
+		}
 	}
 
 	/**


[05/12] flink git commit: [FLINK-6830] [tests] Add StatefulJobSavepointFrom13MigrationITCase

Posted by tz...@apache.org.
[FLINK-6830] [tests] Add StatefulJobSavepointFrom13MigrationITCase


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

Branch: refs/heads/release-1.3
Commit: 7a4d016eff4eccaaa2955e2d41ffba7b3bef17c4
Parents: f74caf7
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Fri Jun 2 17:18:51 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:52:16 2017 +0200

----------------------------------------------------------------------
 .../utils/SavepointMigrationTestBase.java       |   8 ++-
 ...atefulJobSavepointFrom12MigrationITCase.java |  19 +++---
 ...atefulJobSavepointFrom13MigrationITCase.java |  58 +++++++++++++++++++
 .../_metadata                                   | Bin 0 -> 36467 bytes
 .../_metadata                                   | Bin 0 -> 36395 bytes
 5 files changed, 77 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7a4d016e/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 c5672a2..e4004c7 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
@@ -169,7 +169,13 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		final String jobmanagerSavepointPath = ((JobManagerMessages.TriggerSavepointSuccess) savepointResult).savepointPath();
 		LOG.info("Saved savepoint: " + jobmanagerSavepointPath);
 
-		FileUtils.moveFile(new File(new URI(jobmanagerSavepointPath).getPath()), new File(savepointPath));
+		File jobManagerSavepoint = new File(new URI(jobmanagerSavepointPath).getPath());
+		// savepoints were changed to be directories in Flink 1.3
+		if (jobManagerSavepoint.isDirectory()) {
+			FileUtils.moveDirectory(jobManagerSavepoint, new File(savepointPath));
+		} else {
+			FileUtils.moveFile(jobManagerSavepoint, new File(savepointPath));
+		}
 	}
 
 	@SafeVarargs

http://git-wip-us.apache.org/repos/asf/flink/blob/7a4d016e/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 5f03195..4a1d181 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
@@ -76,7 +76,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		// we only test memory state backend yet
 		env.setStateBackend(new MemoryStateBackend());
 		env.enableCheckpointing(500);
 		env.setParallelism(4);
@@ -103,7 +102,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		executeAndSavepoint(
 				env,
-				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
+				"src/test/resources/" + getSavepointPath(),
 				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
 	}
 
@@ -144,7 +143,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		executeAndSavepoint(
 				env,
-				"src/test/resources/stateful-udf-migration-itcase-flink1.2-rocksdb-savepoint",
+				"src/test/resources/" + getRocksDBSavepointPath(),
 				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
 	}
 
@@ -155,7 +154,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setRestartStrategy(RestartStrategies.noRestart());
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		// we only test memory state backend yet
 		env.setStateBackend(new MemoryStateBackend());
 		env.enableCheckpointing(500);
 		env.setParallelism(4);
@@ -182,7 +180,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		restoreAndExecute(
 				env,
-				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
+				getResourceFilename(getSavepointPath()),
 				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
 				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
 				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
@@ -201,7 +199,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 		env.setRestartStrategy(RestartStrategies.noRestart());
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		// we only test memory state backend yet
 		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
 		env.enableCheckpointing(500);
 		env.setParallelism(4);
@@ -228,7 +225,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		restoreAndExecute(
 				env,
-				getResourceFilename("stateful-udf-migration-itcase-flink1.2-rocksdb-savepoint"),
+				getResourceFilename(getRocksDBSavepointPath()),
 				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
 				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
 				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
@@ -241,6 +238,14 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
 	}
 
+	protected String getSavepointPath() {
+		return "stateful-udf-migration-itcase-flink1.2-savepoint";
+	}
+
+	protected String getRocksDBSavepointPath() {
+		return "stateful-udf-migration-itcase-flink1.2-rocksdb-savepoint";
+	}
+
 	private static class LegacyCheckpointedSource
 			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7a4d016e/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom13MigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom13MigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom13MigrationITCase.java
new file mode 100644
index 0000000..a2d3201
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom13MigrationITCase.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.checkpointing.utils;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * This verifies that we can restore a complete job from a Flink 1.3 savepoint.
+ *
+ * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
+ */
+public class StatefulJobSavepointFrom13MigrationITCase extends StatefulJobSavepointFrom12MigrationITCase {
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.3.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink13() throws Exception {
+		testCreateSavepointOnFlink12();
+	}
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.3.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink13WithRocksDB() throws Exception {
+		testCreateSavepointOnFlink12WithRocksDB();
+	}
+
+	@Override
+	protected String getSavepointPath() {
+		return "stateful-udf-migration-itcase-flink1.3-savepoint";
+	}
+
+	@Override
+	protected String getRocksDBSavepointPath() {
+		return "stateful-udf-migration-itcase-flink1.3-rocksdb-savepoint";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a4d016e/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-rocksdb-savepoint/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-rocksdb-savepoint/_metadata b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-rocksdb-savepoint/_metadata
new file mode 100644
index 0000000..8f22bcb
Binary files /dev/null and b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-rocksdb-savepoint/_metadata differ

http://git-wip-us.apache.org/repos/asf/flink/blob/7a4d016e/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-savepoint/_metadata
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-savepoint/_metadata b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-savepoint/_metadata
new file mode 100644
index 0000000..8ca91ec
Binary files /dev/null and b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.3-savepoint/_metadata differ


[07/12] flink git commit: [FLINK-6830] [cep] Port CEPFrom12MigrationTest for Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [cep] Port CEPFrom12MigrationTest for Flink 1.3


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

Branch: refs/heads/release-1.3
Commit: 8dec81c0df0e32ba067d70877715132436e94dd2
Parents: 8f72070
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sat Jun 3 23:11:31 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:57:39 2017 +0200

----------------------------------------------------------------------
 .../cep/operator/CEPFrom12MigrationTest.java    | 605 ------------------
 .../flink/cep/operator/CEPMigrationTest.java    | 635 +++++++++++++++++++
 ...-migration-after-branching-flink1.2-snapshot | Bin 5580 -> 5562 bytes
 ...-migration-after-branching-flink1.3-snapshot | Bin 0 -> 21980 bytes
 ...-single-pattern-afterwards-flink1.2-snapshot | Bin 2326 -> 2326 bytes
 ...-single-pattern-afterwards-flink1.3-snapshot | Bin 0 -> 19770 bytes
 ...ation-starting-new-pattern-flink1.2-snapshot | Bin 5389 -> 5371 bytes
 ...ation-starting-new-pattern-flink1.3-snapshot | Bin 0 -> 21788 bytes
 .../util/migration/MigrationTestUtil.java       |  50 ++
 .../util/migration/MigrationVersion.java        |  43 ++
 10 files changed, 728 insertions(+), 605 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java
deleted file mode 100644
index 0345192..0000000
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java
+++ /dev/null
@@ -1,605 +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.cep.operator;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.cep.Event;
-import org.apache.flink.cep.SubEvent;
-import org.apache.flink.cep.nfa.NFA;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
-import org.apache.flink.cep.pattern.Pattern;
-import org.apache.flink.cep.pattern.conditions.SimpleCondition;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for checking whether CEP operator can restore from snapshots that were done
- * using the Flink 1.2 operator.
- *
- * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
- * the Flink 1.2 branch.
- */
-
-public class CEPFrom12MigrationTest {
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writAfterBranchingPatternSnapshot() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent = new Event(42, "start", 1.0);
-		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
-		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new NFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.open();
-
-			harness.processElement(new StreamRecord<Event>(startEvent, 1));
-			harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
-			harness
-				.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
-			harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
-			harness.processElement(new StreamRecord<Event>(middleEvent2, 3));
-
-			harness.processWatermark(new Watermark(5));
-
-			// do snapshot and save to file
-			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
-			OperatorSnapshotUtil.writeStateHandle(snapshot,
-				"src/test/resources/cep-migration-after-branching-flink1.2-snapshot");
-		} finally {
-			harness.close();
-		}
-	}
-
-	@Test
-	public void testRestoreAfterBranchingPattern() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent = new Event(42, "start", 1.0);
-		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
-		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
-		final Event endEvent = new Event(42, "end", 1.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new NFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-					OperatorSnapshotUtil
-						.getResourceFilename("cep-migration-after-branching-flink1.2-snapshot")));
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(new Event(42, "start", 1.0), 4));
-			harness.processElement(new StreamRecord<>(endEvent, 5));
-
-			harness.processWatermark(new Watermark(20));
-
-			ConcurrentLinkedQueue<Object> result = harness.getOutput();
-
-			// watermark and 2 results
-			assertEquals(3, result.size());
-
-			Object resultObject1 = result.poll();
-			assertTrue(resultObject1 instanceof StreamRecord);
-			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
-			assertTrue(resultRecord1.getValue() instanceof Map);
-
-			Object resultObject2 = result.poll();
-			assertTrue(resultObject2 instanceof StreamRecord);
-			StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
-			assertTrue(resultRecord2.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap1 =
-				(Map<String, List<Event>>) resultRecord1.getValue();
-
-			assertEquals(startEvent, patternMap1.get("start").get(0));
-			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
-			assertEquals(endEvent, patternMap1.get("end").get(0));
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap2 =
-				(Map<String, List<Event>>) resultRecord2.getValue();
-
-			assertEquals(startEvent, patternMap2.get("start").get(0));
-			assertEquals(middleEvent2, patternMap2.get("middle").get(0));
-			assertEquals(endEvent, patternMap2.get("end").get(0));
-
-			// and now go for a checkpoint with the new serializers
-
-			final Event startEvent1 = new Event(42, "start", 2.0);
-			final SubEvent middleEvent3 = new SubEvent(42, "foo", 1.0, 11.0);
-			final Event endEvent1 = new Event(42, "end", 2.0);
-
-			harness.processElement(new StreamRecord<Event>(startEvent1, 21));
-			harness.processElement(new StreamRecord<Event>(middleEvent3, 23));
-
-			// simulate snapshot/restore with some elements in internal sorting queue
-			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
-			harness.close();
-
-			harness = new KeyedOneInputStreamOperatorTestHarness<>(
-				new KeyedCEPPatternOperator<>(
-					Event.createTypeSerializer(),
-					false,
-					IntSerializer.INSTANCE,
-					new NFAFactory(),
-					true),
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-
-			harness.setup();
-			harness.initializeState(snapshot);
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(endEvent1, 25));
-
-			harness.processWatermark(new Watermark(50));
-
-			result = harness.getOutput();
-
-			// watermark and the result
-			assertEquals(2, result.size());
-
-			Object resultObject3 = result.poll();
-			assertTrue(resultObject3 instanceof StreamRecord);
-			StreamRecord<?> resultRecord3 = (StreamRecord<?>) resultObject3;
-			assertTrue(resultRecord3.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap3 =
-				(Map<String, List<Event>>) resultRecord3.getValue();
-
-			assertEquals(startEvent1, patternMap3.get("start").get(0));
-			assertEquals(middleEvent3, patternMap3.get("middle").get(0));
-			assertEquals(endEvent1, patternMap3.get("end").get(0));
-		} finally {
-			harness.close();
-		}
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeStartingNewPatternAfterMigrationSnapshot() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent1 = new Event(42, "start", 1.0);
-		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new NFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.open();
-			harness.processElement(new StreamRecord<Event>(startEvent1, 1));
-			harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
-			harness
-				.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
-			harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
-			harness.processWatermark(new Watermark(5));
-
-			// do snapshot and save to file
-			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
-			OperatorSnapshotUtil.writeStateHandle(snapshot,
-				"src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot");
-		} finally {
-			harness.close();
-		}
-	}
-
-	@Test
-	public void testRestoreStartingNewPatternAfterMigration() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent1 = new Event(42, "start", 1.0);
-		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
-		final Event startEvent2 = new Event(42, "start", 5.0);
-		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
-		final Event endEvent = new Event(42, "end", 1.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new NFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-					OperatorSnapshotUtil.getResourceFilename(
-						"cep-migration-starting-new-pattern-flink1.2-snapshot")));
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(startEvent2, 5));
-			harness.processElement(new StreamRecord<Event>(middleEvent2, 6));
-			harness.processElement(new StreamRecord<>(endEvent, 7));
-
-			harness.processWatermark(new Watermark(20));
-
-			ConcurrentLinkedQueue<Object> result = harness.getOutput();
-
-			// watermark and 3 results
-			assertEquals(4, result.size());
-
-			Object resultObject1 = result.poll();
-			assertTrue(resultObject1 instanceof StreamRecord);
-			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
-			assertTrue(resultRecord1.getValue() instanceof Map);
-
-			Object resultObject2 = result.poll();
-			assertTrue(resultObject2 instanceof StreamRecord);
-			StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
-			assertTrue(resultRecord2.getValue() instanceof Map);
-
-			Object resultObject3 = result.poll();
-			assertTrue(resultObject3 instanceof StreamRecord);
-			StreamRecord<?> resultRecord3 = (StreamRecord<?>) resultObject3;
-			assertTrue(resultRecord3.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap1 =
-				(Map<String, List<Event>>) resultRecord1.getValue();
-
-			assertEquals(startEvent1, patternMap1.get("start").get(0));
-			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
-			assertEquals(endEvent, patternMap1.get("end").get(0));
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap2 =
-				(Map<String, List<Event>>) resultRecord2.getValue();
-
-			assertEquals(startEvent1, patternMap2.get("start").get(0));
-			assertEquals(middleEvent2, patternMap2.get("middle").get(0));
-			assertEquals(endEvent, patternMap2.get("end").get(0));
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap3 =
-				(Map<String, List<Event>>) resultRecord3.getValue();
-
-			assertEquals(startEvent2, patternMap3.get("start").get(0));
-			assertEquals(middleEvent2, patternMap3.get("middle").get(0));
-			assertEquals(endEvent, patternMap3.get("end").get(0));
-
-			// and now go for a checkpoint with the new serializers
-
-			final Event startEvent3 = new Event(42, "start", 2.0);
-			final SubEvent middleEvent3 = new SubEvent(42, "foo", 1.0, 11.0);
-			final Event endEvent1 = new Event(42, "end", 2.0);
-
-			harness.processElement(new StreamRecord<Event>(startEvent3, 21));
-			harness.processElement(new StreamRecord<Event>(middleEvent3, 23));
-
-			// simulate snapshot/restore with some elements in internal sorting queue
-			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
-			harness.close();
-
-			harness = new KeyedOneInputStreamOperatorTestHarness<>(
-				new KeyedCEPPatternOperator<>(
-					Event.createTypeSerializer(),
-					false,
-					IntSerializer.INSTANCE,
-					new NFAFactory(),
-					true),
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-
-			harness.setup();
-			harness.initializeState(snapshot);
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(endEvent1, 25));
-
-			harness.processWatermark(new Watermark(50));
-
-			result = harness.getOutput();
-
-			// watermark and the result
-			assertEquals(2, result.size());
-
-			Object resultObject4 = result.poll();
-			assertTrue(resultObject4 instanceof StreamRecord);
-			StreamRecord<?> resultRecord4 = (StreamRecord<?>) resultObject4;
-			assertTrue(resultRecord4.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap4 =
-				(Map<String, List<Event>>) resultRecord4.getValue();
-
-			assertEquals(startEvent3, patternMap4.get("start").get(0));
-			assertEquals(middleEvent3, patternMap4.get("middle").get(0));
-			assertEquals(endEvent1, patternMap4.get("end").get(0));
-		} finally {
-			harness.close();
-		}
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeSinglePatternAfterMigrationSnapshot() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent1 = new Event(42, "start", 1.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new SinglePatternNFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.open();
-			harness.processWatermark(new Watermark(5));
-
-			// do snapshot and save to file
-			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
-			OperatorSnapshotUtil.writeStateHandle(snapshot,
-				"src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot");
-		} finally {
-			harness.close();
-		}
-	}
-
-
-	@Test
-	public void testSinglePatternAfterMigration() throws Exception {
-
-		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
-			private static final long serialVersionUID = -4873366487571254798L;
-
-			@Override
-			public Integer getKey(Event value) throws Exception {
-				return value.getId();
-			}
-		};
-
-		final Event startEvent1 = new Event(42, "start", 1.0);
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-				new KeyedOneInputStreamOperatorTestHarness<>(
-						new KeyedCEPPatternOperator<>(
-								Event.createTypeSerializer(),
-								false,
-								IntSerializer.INSTANCE,
-								new SinglePatternNFAFactory(),
-								true),
-						keySelector,
-						BasicTypeInfo.INT_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-					OperatorSnapshotUtil.getResourceFilename(
-						"cep-migration-single-pattern-afterwards-flink1.2-snapshot")));
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(startEvent1, 5));
-
-			harness.processWatermark(new Watermark(20));
-
-			ConcurrentLinkedQueue<Object> result = harness.getOutput();
-
-			// watermark and the result
-			assertEquals(2, result.size());
-
-			Object resultObject = result.poll();
-			assertTrue(resultObject instanceof StreamRecord);
-			StreamRecord<?> resultRecord = (StreamRecord<?>) resultObject;
-			assertTrue(resultRecord.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap =
-				(Map<String, List<Event>>) resultRecord.getValue();
-
-			assertEquals(startEvent1, patternMap.get("start").get(0));
-		} finally {
-			harness.close();
-		}
-	}
-
-	private static class SinglePatternNFAFactory implements NFACompiler.NFAFactory<Event> {
-
-		private static final long serialVersionUID = 1173020762472766713L;
-
-		private final boolean handleTimeout;
-
-		private SinglePatternNFAFactory() {
-			this(false);
-		}
-
-		private SinglePatternNFAFactory(boolean handleTimeout) {
-			this.handleTimeout = handleTimeout;
-		}
-
-		@Override
-		public NFA<Event> createNFA() {
-
-			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
-					.within(Time.milliseconds(10L));
-
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
-		}
-	}
-
-	private static class NFAFactory implements NFACompiler.NFAFactory<Event> {
-
-		private static final long serialVersionUID = 1173020762472766713L;
-
-		private final boolean handleTimeout;
-
-		private NFAFactory() {
-			this(false);
-		}
-
-		private NFAFactory(boolean handleTimeout) {
-			this.handleTimeout = handleTimeout;
-		}
-
-		@Override
-		public NFA<Event> createNFA() {
-
-			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
-					.followedBy("middle")
-					.subtype(SubEvent.class)
-					.where(new MiddleFilter())
-					.followedBy("end")
-					.where(new EndFilter())
-					// add a window timeout to test whether timestamps of elements in the
-					// priority queue in CEP operator are correctly checkpointed/restored
-					.within(Time.milliseconds(10L));
-
-			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
-		}
-	}
-
-	private static class StartFilter extends SimpleCondition<Event> {
-		private static final long serialVersionUID = 5726188262756267490L;
-
-		@Override
-		public boolean filter(Event value) throws Exception {
-			return value.getName().equals("start");
-		}
-	}
-
-	private static class MiddleFilter extends SimpleCondition<SubEvent> {
-		private static final long serialVersionUID = 6215754202506583964L;
-
-		@Override
-		public boolean filter(SubEvent value) throws Exception {
-			return value.getVolume() > 5.0;
-		}
-	}
-
-	private static class EndFilter extends SimpleCondition<Event> {
-		private static final long serialVersionUID = 7056763917392056548L;
-
-		@Override
-		public boolean filter(Event value) throws Exception {
-			return value.getName().equals("end");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
new file mode 100644
index 0000000..459a7c5
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
@@ -0,0 +1,635 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.cep.operator;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.cep.Event;
+import org.apache.flink.cep.SubEvent;
+import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for checking whether CEP operator can restore from snapshots that were done
+ * using previous Flink versions.
+ *
+ * <p>For regenerating the binary snapshot file of previous versions you have to run the
+ * {@code write*()} method on the corresponding Flink release-* branch.
+ */
+@RunWith(Parameterized.class)
+public class CEPMigrationTest {
+
+	/**
+	 * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+	 * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints
+	 */
+	private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+	private final MigrationVersion migrateVersion;
+
+	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
+	public static Collection<MigrationVersion> parameters () {
+		return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3);
+	}
+
+	public CEPMigrationTest(MigrationVersion migrateVersion) {
+		this.migrateVersion = migrateVersion;
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeAfterBranchingPatternSnapshot() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new NFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+			harness.open();
+
+			harness.processElement(new StreamRecord<Event>(startEvent, 1));
+			harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
+			harness
+				.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
+			harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
+			harness.processElement(new StreamRecord<Event>(middleEvent2, 3));
+
+			harness.processWatermark(new Watermark(5));
+
+			// do snapshot and save to file
+			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
+			OperatorSnapshotUtil.writeStateHandle(snapshot,
+				"src/test/resources/cep-migration-after-branching-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		} finally {
+			harness.close();
+		}
+	}
+
+	@Test
+	public void testRestoreAfterBranchingPattern() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
+		final Event endEvent = new Event(42, "end", 1.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new NFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+
+			MigrationTestUtil.restoreFromSnapshot(
+				harness,
+				OperatorSnapshotUtil.getResourceFilename("cep-migration-after-branching-flink" + migrateVersion + "-snapshot"),
+				migrateVersion);
+
+			harness.open();
+
+			harness.processElement(new StreamRecord<>(new Event(42, "start", 1.0), 4));
+			harness.processElement(new StreamRecord<>(endEvent, 5));
+
+			harness.processWatermark(new Watermark(20));
+
+			ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+			// watermark and 2 results
+			assertEquals(3, result.size());
+
+			Object resultObject1 = result.poll();
+			assertTrue(resultObject1 instanceof StreamRecord);
+			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
+			assertTrue(resultRecord1.getValue() instanceof Map);
+
+			Object resultObject2 = result.poll();
+			assertTrue(resultObject2 instanceof StreamRecord);
+			StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
+			assertTrue(resultRecord2.getValue() instanceof Map);
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap1 =
+				(Map<String, List<Event>>) resultRecord1.getValue();
+
+			assertEquals(startEvent, patternMap1.get("start").get(0));
+			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
+			assertEquals(endEvent, patternMap1.get("end").get(0));
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap2 =
+				(Map<String, List<Event>>) resultRecord2.getValue();
+
+			assertEquals(startEvent, patternMap2.get("start").get(0));
+			assertEquals(middleEvent2, patternMap2.get("middle").get(0));
+			assertEquals(endEvent, patternMap2.get("end").get(0));
+
+			// and now go for a checkpoint with the new serializers
+
+			final Event startEvent1 = new Event(42, "start", 2.0);
+			final SubEvent middleEvent3 = new SubEvent(42, "foo", 1.0, 11.0);
+			final Event endEvent1 = new Event(42, "end", 2.0);
+
+			harness.processElement(new StreamRecord<Event>(startEvent1, 21));
+			harness.processElement(new StreamRecord<Event>(middleEvent3, 23));
+
+			// simulate snapshot/restore with some elements in internal sorting queue
+			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
+			harness.close();
+
+			harness = new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					IntSerializer.INSTANCE,
+					new NFAFactory(),
+					true),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+			harness.setup();
+			harness.initializeState(snapshot);
+			harness.open();
+
+			harness.processElement(new StreamRecord<>(endEvent1, 25));
+
+			harness.processWatermark(new Watermark(50));
+
+			result = harness.getOutput();
+
+			// watermark and the result
+			assertEquals(2, result.size());
+
+			Object resultObject3 = result.poll();
+			assertTrue(resultObject3 instanceof StreamRecord);
+			StreamRecord<?> resultRecord3 = (StreamRecord<?>) resultObject3;
+			assertTrue(resultRecord3.getValue() instanceof Map);
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap3 =
+				(Map<String, List<Event>>) resultRecord3.getValue();
+
+			assertEquals(startEvent1, patternMap3.get("start").get(0));
+			assertEquals(middleEvent3, patternMap3.get("middle").get(0));
+			assertEquals(endEvent1, patternMap3.get("end").get(0));
+		} finally {
+			harness.close();
+		}
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeStartingNewPatternAfterMigrationSnapshot() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new NFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+			harness.open();
+			harness.processElement(new StreamRecord<Event>(startEvent1, 1));
+			harness.processElement(new StreamRecord<Event>(new Event(42, "foobar", 1.0), 2));
+			harness
+				.processElement(new StreamRecord<Event>(new SubEvent(42, "barfoo", 1.0, 5.0), 3));
+			harness.processElement(new StreamRecord<Event>(middleEvent1, 2));
+			harness.processWatermark(new Watermark(5));
+
+			// do snapshot and save to file
+			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
+			OperatorSnapshotUtil.writeStateHandle(snapshot,
+				"src/test/resources/cep-migration-starting-new-pattern-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		} finally {
+			harness.close();
+		}
+	}
+
+	@Test
+	public void testRestoreStartingNewPatternAfterMigration() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+		final SubEvent middleEvent1 = new SubEvent(42, "foo1", 1.0, 10.0);
+		final Event startEvent2 = new Event(42, "start", 5.0);
+		final SubEvent middleEvent2 = new SubEvent(42, "foo2", 2.0, 10.0);
+		final Event endEvent = new Event(42, "end", 1.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new NFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+
+			MigrationTestUtil.restoreFromSnapshot(
+				harness,
+				OperatorSnapshotUtil.getResourceFilename("cep-migration-starting-new-pattern-flink" + migrateVersion + "-snapshot"),
+				migrateVersion);
+
+			harness.open();
+
+			harness.processElement(new StreamRecord<>(startEvent2, 5));
+			harness.processElement(new StreamRecord<Event>(middleEvent2, 6));
+			harness.processElement(new StreamRecord<>(endEvent, 7));
+
+			harness.processWatermark(new Watermark(20));
+
+			ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+			// watermark and 3 results
+			assertEquals(4, result.size());
+
+			Object resultObject1 = result.poll();
+			assertTrue(resultObject1 instanceof StreamRecord);
+			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
+			assertTrue(resultRecord1.getValue() instanceof Map);
+
+			Object resultObject2 = result.poll();
+			assertTrue(resultObject2 instanceof StreamRecord);
+			StreamRecord<?> resultRecord2 = (StreamRecord<?>) resultObject2;
+			assertTrue(resultRecord2.getValue() instanceof Map);
+
+			Object resultObject3 = result.poll();
+			assertTrue(resultObject3 instanceof StreamRecord);
+			StreamRecord<?> resultRecord3 = (StreamRecord<?>) resultObject3;
+			assertTrue(resultRecord3.getValue() instanceof Map);
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap1 =
+				(Map<String, List<Event>>) resultRecord1.getValue();
+
+			assertEquals(startEvent1, patternMap1.get("start").get(0));
+			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
+			assertEquals(endEvent, patternMap1.get("end").get(0));
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap2 =
+				(Map<String, List<Event>>) resultRecord2.getValue();
+
+			assertEquals(startEvent1, patternMap2.get("start").get(0));
+			assertEquals(middleEvent2, patternMap2.get("middle").get(0));
+			assertEquals(endEvent, patternMap2.get("end").get(0));
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap3 =
+				(Map<String, List<Event>>) resultRecord3.getValue();
+
+			assertEquals(startEvent2, patternMap3.get("start").get(0));
+			assertEquals(middleEvent2, patternMap3.get("middle").get(0));
+			assertEquals(endEvent, patternMap3.get("end").get(0));
+
+			// and now go for a checkpoint with the new serializers
+
+			final Event startEvent3 = new Event(42, "start", 2.0);
+			final SubEvent middleEvent3 = new SubEvent(42, "foo", 1.0, 11.0);
+			final Event endEvent1 = new Event(42, "end", 2.0);
+
+			harness.processElement(new StreamRecord<Event>(startEvent3, 21));
+			harness.processElement(new StreamRecord<Event>(middleEvent3, 23));
+
+			// simulate snapshot/restore with some elements in internal sorting queue
+			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
+			harness.close();
+
+			harness = new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					IntSerializer.INSTANCE,
+					new NFAFactory(),
+					true),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+			harness.setup();
+			harness.initializeState(snapshot);
+			harness.open();
+
+			harness.processElement(new StreamRecord<>(endEvent1, 25));
+
+			harness.processWatermark(new Watermark(50));
+
+			result = harness.getOutput();
+
+			// watermark and the result
+			assertEquals(2, result.size());
+
+			Object resultObject4 = result.poll();
+			assertTrue(resultObject4 instanceof StreamRecord);
+			StreamRecord<?> resultRecord4 = (StreamRecord<?>) resultObject4;
+			assertTrue(resultRecord4.getValue() instanceof Map);
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap4 =
+				(Map<String, List<Event>>) resultRecord4.getValue();
+
+			assertEquals(startEvent3, patternMap4.get("start").get(0));
+			assertEquals(middleEvent3, patternMap4.get("middle").get(0));
+			assertEquals(endEvent1, patternMap4.get("end").get(0));
+		} finally {
+			harness.close();
+		}
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeSinglePatternAfterMigrationSnapshot() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new SinglePatternNFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+			harness.open();
+			harness.processWatermark(new Watermark(5));
+
+			// do snapshot and save to file
+			OperatorStateHandles snapshot = harness.snapshot(0L, 0L);
+			OperatorSnapshotUtil.writeStateHandle(snapshot,
+				"src/test/resources/cep-migration-single-pattern-afterwards-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		} finally {
+			harness.close();
+		}
+	}
+
+
+	@Test
+	public void testSinglePatternAfterMigration() throws Exception {
+
+		KeySelector<Event, Integer> keySelector = new KeySelector<Event, Integer>() {
+			private static final long serialVersionUID = -4873366487571254798L;
+
+			@Override
+			public Integer getKey(Event value) throws Exception {
+				return value.getId();
+			}
+		};
+
+		final Event startEvent1 = new Event(42, "start", 1.0);
+
+		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								IntSerializer.INSTANCE,
+								new SinglePatternNFAFactory(),
+								true),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		try {
+			harness.setup();
+
+			MigrationTestUtil.restoreFromSnapshot(
+				harness,
+				OperatorSnapshotUtil.getResourceFilename("cep-migration-single-pattern-afterwards-flink" + migrateVersion + "-snapshot"),
+				migrateVersion);
+
+			harness.open();
+
+			harness.processElement(new StreamRecord<>(startEvent1, 5));
+
+			harness.processWatermark(new Watermark(20));
+
+			ConcurrentLinkedQueue<Object> result = harness.getOutput();
+
+			// watermark and the result
+			assertEquals(2, result.size());
+
+			Object resultObject = result.poll();
+			assertTrue(resultObject instanceof StreamRecord);
+			StreamRecord<?> resultRecord = (StreamRecord<?>) resultObject;
+			assertTrue(resultRecord.getValue() instanceof Map);
+
+			@SuppressWarnings("unchecked")
+			Map<String, List<Event>> patternMap =
+				(Map<String, List<Event>>) resultRecord.getValue();
+
+			assertEquals(startEvent1, patternMap.get("start").get(0));
+		} finally {
+			harness.close();
+		}
+	}
+
+	private static class SinglePatternNFAFactory implements NFACompiler.NFAFactory<Event> {
+
+		private static final long serialVersionUID = 1173020762472766713L;
+
+		private final boolean handleTimeout;
+
+		private SinglePatternNFAFactory() {
+			this(false);
+		}
+
+		private SinglePatternNFAFactory(boolean handleTimeout) {
+			this.handleTimeout = handleTimeout;
+		}
+
+		@Override
+		public NFA<Event> createNFA() {
+
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
+					.within(Time.milliseconds(10L));
+
+			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+		}
+	}
+
+	private static class NFAFactory implements NFACompiler.NFAFactory<Event> {
+
+		private static final long serialVersionUID = 1173020762472766713L;
+
+		private final boolean handleTimeout;
+
+		private NFAFactory() {
+			this(false);
+		}
+
+		private NFAFactory(boolean handleTimeout) {
+			this.handleTimeout = handleTimeout;
+		}
+
+		@Override
+		public NFA<Event> createNFA() {
+
+			Pattern<Event, ?> pattern = Pattern.<Event>begin("start").where(new StartFilter())
+					.followedByAny("middle")
+					.subtype(SubEvent.class)
+					.where(new MiddleFilter())
+					.followedByAny("end")
+					.where(new EndFilter())
+					// add a window timeout to test whether timestamps of elements in the
+					// priority queue in CEP operator are correctly checkpointed/restored
+					.within(Time.milliseconds(10L));
+
+			return NFACompiler.compile(pattern, Event.createTypeSerializer(), handleTimeout);
+		}
+	}
+
+	private static class StartFilter extends SimpleCondition<Event> {
+		private static final long serialVersionUID = 5726188262756267490L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("start");
+		}
+	}
+
+	private static class MiddleFilter extends SimpleCondition<SubEvent> {
+		private static final long serialVersionUID = 6215754202506583964L;
+
+		@Override
+		public boolean filter(SubEvent value) throws Exception {
+			return value.getVolume() > 5.0;
+		}
+	}
+
+	private static class EndFilter extends SimpleCondition<Event> {
+		private static final long serialVersionUID = 7056763917392056548L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("end");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.2-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.2-snapshot
index 6775f2a..4eb4e44 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.2-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.2-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.3-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.3-snapshot
new file mode 100644
index 0000000..bee7aeb
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-after-branching-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot
index f63b7dd..64e973e 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.2-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.3-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.3-snapshot
new file mode 100644
index 0000000..faa0bf9
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-single-pattern-afterwards-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot
index 8e0fd27..5e48241 100644
Binary files a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.3-snapshot b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.3-snapshot
new file mode 100644
index 0000000..1a2f388
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.3-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
new file mode 100644
index 0000000..f723b34
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.migration;
+
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+
+/**
+ * Utility methods for testing snapshot migrations.
+ */
+public class MigrationTestUtil {
+
+	/**
+	 * Restore from a snapshot taken with an older Flink version.
+	 *
+	 * @param testHarness the test harness to restore the snapshot to.
+	 * @param snapshotPath the absolute path to the snapshot.
+	 * @param snapshotFlinkVersion the Flink version of the snapshot.
+	 *
+	 * @throws Exception
+	 */
+	public static void restoreFromSnapshot(
+			AbstractStreamOperatorTestHarness<?> testHarness,
+			String snapshotPath,
+			MigrationVersion snapshotFlinkVersion) throws Exception {
+
+		if (snapshotFlinkVersion == MigrationVersion.v1_1) {
+			// Flink 1.1 snapshots should be read using the legacy restore method
+			testHarness.initializeStateFromLegacyCheckpoint(snapshotPath);
+		} else {
+			testHarness.initializeState(OperatorSnapshotUtil.readStateHandle(snapshotPath));
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8dec81c0/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationVersion.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationVersion.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationVersion.java
new file mode 100644
index 0000000..3e7998d
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationVersion.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.util.migration;
+
+/**
+ * Enumeration for Flink versions, used in migration integration tests
+ * to indicate the migrated snapshot version.
+ */
+public enum MigrationVersion {
+
+	// NOTE: the version strings must not change,
+	// as they are used to locate snapshot file paths
+	v1_1("1.1"),
+	v1_2("1.2"),
+	v1_3("1.3");
+
+	private String versionStr;
+
+	MigrationVersion(String versionStr) {
+		this.versionStr = versionStr;
+	}
+
+	@Override
+	public String toString() {
+		return versionStr;
+	}
+}


[04/12] flink git commit: [FLINK-6808] Implement snapshotConfiguration/ensureCompatibility for CoGroupedStreams.UnionSerializer

Posted by tz...@apache.org.
[FLINK-6808] Implement snapshotConfiguration/ensureCompatibility for CoGroupedStreams.UnionSerializer

This closes #4052.


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

Branch: refs/heads/release-1.3
Commit: f74caf7062b1cc23a704f8f8b8171be430b60807
Parents: 1d89dd0
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Fri Jun 2 15:15:32 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 18:52:08 2017 +0200

----------------------------------------------------------------------
 .../api/datastream/CoGroupedStreams.java        | 60 ++++++++++++++++++--
 flink-tests/pom.xml                             |  1 +
 .../streaming/runtime/CoGroupJoinITCase.java    | 47 +++++++++++++++
 3 files changed, 103 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f74caf70/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
index f0c3dc2..ba26623 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -30,10 +30,15 @@ import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -76,7 +81,7 @@ public class CoGroupedStreams<T1, T2> {
 	private final DataStream<T2> input2;
 
 	/**
-	 * Creates new CoGroped data streams, which are the first step towards building a streaming
+	 * Creates new CoGrouped data streams, which are the first step towards building a streaming
 	 * co-group.
 	 *
 	 * @param input1 The first data stream.
@@ -442,8 +447,7 @@ public class CoGroupedStreams<T1, T2> {
 		private final TypeSerializer<T1> oneSerializer;
 		private final TypeSerializer<T2> twoSerializer;
 
-		public UnionSerializer(TypeSerializer<T1> oneSerializer,
-				TypeSerializer<T2> twoSerializer) {
+		public UnionSerializer(TypeSerializer<T1> oneSerializer, TypeSerializer<T2> twoSerializer) {
 			this.oneSerializer = oneSerializer;
 			this.twoSerializer = twoSerializer;
 		}
@@ -552,12 +556,58 @@ public class CoGroupedStreams<T1, T2> {
 
 		@Override
 		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			throw new UnsupportedOperationException("This serializer is not registered for managed state.");
+			return new UnionSerializerConfigSnapshot<>(oneSerializer, twoSerializer);
 		}
 
 		@Override
 		public CompatibilityResult<TaggedUnion<T1, T2>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			throw new UnsupportedOperationException("This serializer is not registered for managed state.");
+			if (configSnapshot instanceof UnionSerializerConfigSnapshot) {
+				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs =
+					((UnionSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
+
+				CompatibilityResult<T1> oneSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult(
+					previousSerializersAndConfigs.get(0).f0,
+					UnloadableDummyTypeSerializer.class,
+					previousSerializersAndConfigs.get(0).f1,
+					oneSerializer);
+
+				CompatibilityResult<T2> twoSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult(
+					previousSerializersAndConfigs.get(1).f0,
+					UnloadableDummyTypeSerializer.class,
+					previousSerializersAndConfigs.get(1).f1,
+					twoSerializer);
+
+				if (!oneSerializerCompatResult.isRequiresMigration() && !twoSerializerCompatResult.isRequiresMigration()) {
+					return CompatibilityResult.compatible();
+				} else if (oneSerializerCompatResult.getConvertDeserializer() != null && twoSerializerCompatResult.getConvertDeserializer() != null) {
+					return CompatibilityResult.requiresMigration(
+						new UnionSerializer<>(
+							new TypeDeserializerAdapter<>(oneSerializerCompatResult.getConvertDeserializer()),
+							new TypeDeserializerAdapter<>(twoSerializerCompatResult.getConvertDeserializer())));
+				}
+			}
+
+			return CompatibilityResult.requiresMigration();
+		}
+	}
+
+	/**
+	 * The {@link TypeSerializerConfigSnapshot} for the {@link UnionSerializer}.
+	 */
+	public static class UnionSerializerConfigSnapshot<T1, T2> extends CompositeTypeSerializerConfigSnapshot {
+
+		private static final int VERSION = 1;
+
+		/** This empty nullary constructor is required for deserializing the configuration. */
+		public UnionSerializerConfigSnapshot() {}
+
+		public UnionSerializerConfigSnapshot(TypeSerializer<T1> oneSerializer, TypeSerializer<T2> twoSerializer) {
+			super(oneSerializer, twoSerializer);
+		}
+
+		@Override
+		public int getVersion() {
+			return VERSION;
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f74caf70/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 62feff3..017c213 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -115,6 +115,7 @@ under the License.
 			<artifactId>flink-streaming-java_2.10</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
+			<type>test-jar</type>
 		</dependency>
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/f74caf70/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 da3de3d..a82b965 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
@@ -19,18 +19,24 @@ package org.apache.flink.test.streaming.runtime;
 
 import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 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.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.CoGroupedStreams;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
 import org.apache.flink.streaming.api.windowing.time.Time;
+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;
@@ -324,6 +330,47 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
 		Assert.assertEquals(expectedResult, testResults);
 	}
 
+	/**
+	 * Verifies that pipelines including {@link CoGroupedStreams} can be checkpointed properly,
+	 * which includes snapshotting configurations of any involved serializers.
+	 *
+	 * @see <a href="https://issues.apache.org/jira/browse/FLINK-6808">FLINK-6808</a>
+	 */
+	@Test
+	public void testCoGroupOperatorWithCheckpoint() throws Exception {
+
+		// generate an operator for the co-group operation
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(1);
+
+		DataStream<Tuple2<String, Integer>> source1 = env.fromElements(Tuple2.of("a", 0), Tuple2.of("b", 3));
+		DataStream<Tuple2<String, Integer>> source2 = env.fromElements(Tuple2.of("a", 1), Tuple2.of("b", 6));
+
+		DataStream<String> coGroupWindow = 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>() {
+				@Override
+				public void coGroup(Iterable<Tuple2<String, Integer>> first,
+									Iterable<Tuple2<String, Integer>> second,
+									Collector<String> out) throws Exception {
+					out.collect(first + ":" + second);
+				}
+			});
+
+		OneInputTransformation<Tuple2<String, Integer>, String> transform = (OneInputTransformation<Tuple2<String, Integer>, String>) coGroupWindow.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, String> operator = transform.getOperator();
+
+		// wrap the operator in the test harness, and perform a snapshot
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, String> testHarness =
+			new KeyedOneInputStreamOperatorTestHarness<>(operator, new Tuple2KeyExtractor(), BasicTypeInfo.STRING_TYPE_INFO);
+
+		testHarness.open();
+		testHarness.snapshot(0L, 0L);
+	}
+
 	private static class Tuple2TimestampExtractor implements AssignerWithPunctuatedWatermarks<Tuple2<String, Integer>> {
 		
 		@Override


[08/12] flink git commit: [FLINK-6830] [fileSink] Port BucketingSinkFrom12MigrationTest for Flink 1.3

Posted by tz...@apache.org.
[FLINK-6830] [fileSink] Port BucketingSinkFrom12MigrationTest for Flink 1.3


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

Branch: refs/heads/release-1.3
Commit: 0544b44808d3680fa22695f23d69e341820afde9
Parents: 8dec81c
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sat Jun 3 23:45:22 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Wed Jun 7 19:06:15 2017 +0200

----------------------------------------------------------------------
 .../BucketingSinkFrom12MigrationTest.java       | 223 ----------------
 .../bucketing/BucketingSinkMigrationTest.java   | 262 +++++++++++++++++++
 ...keting-sink-migration-test-flink1.3-snapshot | Bin 0 -> 1862 bytes
 3 files changed, 262 insertions(+), 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0544b448/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java
deleted file mode 100644
index 350b7b4..0000000
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.io.FileUtils;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OperatorSnapshotUtil;
-import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Tests for checking whether {@link BucketingSink} can restore from snapshots that were done
- * using the Flink 1.2 {@link BucketingSink}.
- *
- * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
- * the Flink 1.2 branch.
- */
-
-public class BucketingSinkFrom12MigrationTest {
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	private static final String PART_PREFIX = "part";
-	private static final String PENDING_SUFFIX = ".pending";
-	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
-	private static final String VALID_LENGTH_SUFFIX = ".valid";
-
-	/**
-	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
-	 */
-	@Ignore
-	@Test
-	public void writeSnapshot() throws Exception {
-
-		final File outDir = tempFolder.newFolder();
-
-		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
-			.setWriter(new StringWriter<String>())
-			.setBatchSize(5)
-			.setPartPrefix(PART_PREFIX)
-			.setInProgressPrefix("")
-			.setPendingPrefix("")
-			.setValidLengthPrefix("")
-			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
-			.setPendingSuffix(PENDING_SUFFIX)
-			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness =
-			new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
-
-		testHarness.setup();
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>("test1", 0L));
-		testHarness.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness.processElement(new StreamRecord<>("test3", 0L));
-		testHarness.processElement(new StreamRecord<>("test4", 0L));
-		testHarness.processElement(new StreamRecord<>("test5", 0L));
-
-		checkFs(outDir, 1, 4, 0, 0);
-
-		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
-
-		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot");
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestore() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		ValidatingBucketingSink<String> sink = (ValidatingBucketingSink<String>) new ValidatingBucketingSink<String>(outDir.getAbsolutePath())
-			.setWriter(new StringWriter<String>())
-			.setBatchSize(5)
-			.setPartPrefix(PART_PREFIX)
-			.setInProgressPrefix("")
-			.setPendingPrefix("")
-			.setValidLengthPrefix("")
-			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
-			.setPendingSuffix(PENDING_SUFFIX)
-			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
-
-		OneInputStreamOperatorTestHarness<String, Object> testHarness = new OneInputStreamOperatorTestHarness<>(
-			new StreamSink<>(sink), 10, 1, 0);
-		testHarness.setup();
-		testHarness.initializeState(
-				OperatorSnapshotUtil.readStateHandle(
-						OperatorSnapshotUtil.getResourceFilename("bucketing-sink-migration-test-flink1.2-snapshot")));
-		testHarness.open();
-
-		assertTrue(sink.initializeCalled);
-
-		testHarness.processElement(new StreamRecord<>("test1", 0L));
-		testHarness.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness.close();
-	}
-
-	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
-		int inProg = 0;
-		int pend = 0;
-		int compl = 0;
-		int val = 0;
-
-		for (File file: FileUtils.listFiles(outDir, null, true)) {
-			if (file.getAbsolutePath().endsWith("crc")) {
-				continue;
-			}
-			String path = file.getPath();
-			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
-				inProg++;
-			} else if (path.endsWith(PENDING_SUFFIX)) {
-				pend++;
-			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
-				val++;
-			} else if (path.contains(PART_PREFIX)) {
-				compl++;
-			}
-		}
-
-		Assert.assertEquals(inprogress, inProg);
-		Assert.assertEquals(pending, pend);
-		Assert.assertEquals(completed, compl);
-		Assert.assertEquals(valid, val);
-	}
-
-	static class ValidatingBucketingSink<T> extends BucketingSink<T> {
-
-		private static final long serialVersionUID = -4263974081712009141L;
-
-		public boolean initializeCalled = false;
-
-		ValidatingBucketingSink(String basePath) {
-			super(basePath);
-		}
-
-		/**
-		 * The actual paths in this depend on the binary checkpoint so it you update this the paths
-		 * here have to be updated as well.
-		 */
-		@Override
-		public void initializeState(FunctionInitializationContext context) throws Exception {
-			OperatorStateStore stateStore = context.getOperatorStateStore();
-
-			ListState<State<T>> restoredBucketStates = stateStore.getSerializableListState("bucket-states");
-
-			if (context.isRestored()) {
-
-				for (State<T> states : restoredBucketStates.get()) {
-					for (String bucketPath : states.bucketStates.keySet()) {
-						BucketState state = states.getBucketState(new Path(bucketPath));
-						String current = state.currentFile;
-						long validLength = state.currentFileValidLength;
-
-						Assert.assertEquals("/var/folders/v_/ry2wp5fx0y7c1rvr41xy9_700000gn/T/junit9160378385359106772/junit479663758539998903/1970-01-01--01/part-0-4", current);
-						Assert.assertEquals(6, validLength);
-
-						List<String> pendingFiles = state.pendingFiles;
-						assertTrue(pendingFiles.isEmpty());
-
-						final Map<Long, List<String>> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint;
-						Assert.assertEquals(1, pendingFilesPerCheckpoint.size());
-
-						for (Map.Entry<Long, List<String>> entry: pendingFilesPerCheckpoint.entrySet()) {
-							long checkpoint = entry.getKey();
-							List<String> files = entry.getValue();
-
-							Assert.assertEquals(0L, checkpoint);
-							Assert.assertEquals(4, files.size());
-
-							for (int i = 0; i < 4; i++) {
-								Assert.assertEquals(
-										"/var/folders/v_/ry2wp5fx0y7c1rvr41xy9_700000gn/T/junit9160378385359106772/junit479663758539998903/1970-01-01--01/part-0-" + i,
-										files.get(i));
-							}
-						}
-					}
-				}
-			}
-
-			initializeCalled = true;
-			super.initializeState(context);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0544b448/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkMigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkMigrationTest.java
new file mode 100644
index 0000000..d3383f3
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkMigrationTest.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs.bucketing;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.connectors.fs.StringWriter;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Tests for checking whether {@link BucketingSink} can restore from snapshots that were done
+ * using previous Flink versions' {@link BucketingSink}.
+ *
+ * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
+ * the corresponding Flink release-* branch.
+ */
+@RunWith(Parameterized.class)
+public class BucketingSinkMigrationTest {
+
+	/**
+	 * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+	 * TODO and remove all @Ignore annotations on write*Snapshot() methods to generate savepoints
+	 */
+	private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	private static final String PART_PREFIX = "part";
+	private static final String PENDING_SUFFIX = ".pending";
+	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
+	private static final String VALID_LENGTH_SUFFIX = ".valid";
+
+	@Parameterized.Parameters(name = "Migration Savepoint / Bucket Files Prefix: {0}")
+	public static Collection<Tuple2<MigrationVersion, String>> parameters () {
+		return Arrays.asList(
+			Tuple2.of(MigrationVersion.v1_2, "/var/folders/v_/ry2wp5fx0y7c1rvr41xy9_700000gn/T/junit9160378385359106772/junit479663758539998903/1970-01-01--01/part-0-"),
+			Tuple2.of(MigrationVersion.v1_3, "/var/folders/tv/b_1d8fvx23dgk1_xs8db_95h0000gn/T/junit4273542175898623023/junit3801102997056424640/1970-01-01--01/part-0-"));
+	}
+
+	private final MigrationVersion testMigrateVersion;
+	private final String expectedBucketFilesPrefix;
+
+	public BucketingSinkMigrationTest(Tuple2<MigrationVersion, String> migrateVersionAndExpectedBucketFilesPrefix) {
+		this.testMigrateVersion = migrateVersionAndExpectedBucketFilesPrefix.f0;
+		this.expectedBucketFilesPrefix = migrateVersionAndExpectedBucketFilesPrefix.f1;
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
+	 */
+	@Ignore
+	@Test
+	public void writeSnapshot() throws Exception {
+
+		final File outDir = tempFolder.newFolder();
+
+		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
+			.setWriter(new StringWriter<String>())
+			.setBatchSize(5)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness =
+			new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
+
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.processElement(new StreamRecord<>("test1", 0L));
+		testHarness.processElement(new StreamRecord<>("test2", 0L));
+
+		checkFs(outDir, 1, 1, 0, 0);
+
+		testHarness.processElement(new StreamRecord<>("test3", 0L));
+		testHarness.processElement(new StreamRecord<>("test4", 0L));
+		testHarness.processElement(new StreamRecord<>("test5", 0L));
+
+		checkFs(outDir, 1, 4, 0, 0);
+
+		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
+
+		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/bucketing-sink-migration-test-flink" + flinkGenerateSavepointVersion + "-snapshot");
+		testHarness.close();
+	}
+
+	@Test
+	public void testRestore() throws Exception {
+		final File outDir = tempFolder.newFolder();
+
+		ValidatingBucketingSink<String> sink = (ValidatingBucketingSink<String>)
+				new ValidatingBucketingSink<String>(outDir.getAbsolutePath(), expectedBucketFilesPrefix)
+			.setWriter(new StringWriter<String>())
+			.setBatchSize(5)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = new OneInputStreamOperatorTestHarness<>(
+			new StreamSink<>(sink), 10, 1, 0);
+		testHarness.setup();
+
+		MigrationTestUtil.restoreFromSnapshot(
+			testHarness,
+			OperatorSnapshotUtil.getResourceFilename(
+				"bucketing-sink-migration-test-flink" + testMigrateVersion + "-snapshot"),
+			testMigrateVersion);
+
+		testHarness.open();
+
+		assertTrue(sink.initializeCalled);
+
+		testHarness.processElement(new StreamRecord<>("test1", 0L));
+		testHarness.processElement(new StreamRecord<>("test2", 0L));
+
+		checkFs(outDir, 1, 1, 0, 0);
+
+		testHarness.close();
+	}
+
+	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
+		int inProg = 0;
+		int pend = 0;
+		int compl = 0;
+		int val = 0;
+
+		for (File file: FileUtils.listFiles(outDir, null, true)) {
+			if (file.getAbsolutePath().endsWith("crc")) {
+				continue;
+			}
+			String path = file.getPath();
+			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
+				inProg++;
+			} else if (path.endsWith(PENDING_SUFFIX)) {
+				pend++;
+			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
+				val++;
+			} else if (path.contains(PART_PREFIX)) {
+				compl++;
+			}
+		}
+
+		Assert.assertEquals(inprogress, inProg);
+		Assert.assertEquals(pending, pend);
+		Assert.assertEquals(completed, compl);
+		Assert.assertEquals(valid, val);
+	}
+
+	static class ValidatingBucketingSink<T> extends BucketingSink<T> {
+
+		private static final long serialVersionUID = -4263974081712009141L;
+
+		public boolean initializeCalled = false;
+
+		private final String expectedBucketFilesPrefix;
+
+		ValidatingBucketingSink(String basePath, String expectedBucketFilesPrefix) {
+			super(basePath);
+			this.expectedBucketFilesPrefix = expectedBucketFilesPrefix;
+		}
+
+		/**
+		 * The actual paths in this depend on the binary checkpoint so it you update this the paths
+		 * here have to be updated as well.
+		 */
+		@Override
+		public void initializeState(FunctionInitializationContext context) throws Exception {
+			OperatorStateStore stateStore = context.getOperatorStateStore();
+
+			ListState<State<T>> restoredBucketStates = stateStore.getSerializableListState("bucket-states");
+
+			if (context.isRestored()) {
+
+				for (State<T> states : restoredBucketStates.get()) {
+					for (String bucketPath : states.bucketStates.keySet()) {
+						BucketState state = states.getBucketState(new Path(bucketPath));
+						String current = state.currentFile;
+						long validLength = state.currentFileValidLength;
+
+						Assert.assertEquals(expectedBucketFilesPrefix + "4", current);
+						Assert.assertEquals(6, validLength);
+
+						List<String> pendingFiles = state.pendingFiles;
+						assertTrue(pendingFiles.isEmpty());
+
+						final Map<Long, List<String>> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint;
+						Assert.assertEquals(1, pendingFilesPerCheckpoint.size());
+
+						for (Map.Entry<Long, List<String>> entry: pendingFilesPerCheckpoint.entrySet()) {
+							long checkpoint = entry.getKey();
+							List<String> files = entry.getValue();
+
+							Assert.assertEquals(0L, checkpoint);
+							Assert.assertEquals(4, files.size());
+
+							for (int i = 0; i < 4; i++) {
+								Assert.assertEquals(
+										expectedBucketFilesPrefix + i,
+										files.get(i));
+							}
+						}
+					}
+				}
+			}
+
+			initializeCalled = true;
+			super.initializeState(context);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0544b448/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.3-snapshot b/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.3-snapshot
new file mode 100644
index 0000000..765e8bf
Binary files /dev/null and b/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.3-snapshot differ