You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2017/05/03 11:51:57 UTC

[01/11] flink git commit: [FLINK-6353] Fix legacy user-state restore from 1.2

Repository: flink
Updated Branches:
  refs/heads/release-1.2 fdb3f65f2 -> 852a710b4


[FLINK-6353] Fix legacy user-state restore from 1.2

State that was checkpointed using Checkpointed (on a user function)
could not be restored using CheckpointedRestoring when the savepoint was
done on Flink 1.2. The reason was an overzealous check in
AbstractUdfStreamOperator that only restores from "legacy" operator
state using CheckpointedRestoring when the stream is a Migration stream.

This removes that check but we still need to make sure to read away the
byte that indicates whether there is legacy state, which is written when
we're restoring from a Flink 1.1 savepoint.

After this fix, the procedure for a user to migrate a user function away
from the Checkpointed interface is this:

 - Perform savepoint with user function still implementing Checkpointed,
   shutdown job
 - Change user function to implement CheckpointedRestoring
 - Restore from previous savepoint, user function has to somehow move
   the state that is restored using CheckpointedRestoring to another
   type of state, .e.g operator state, using the OperatorStateStore.
 - Perform another savepoint, shutdown job
 - Remove CheckpointedRestoring interface from user function
 - Restore from the second savepoint
 - Done.

If the CheckpointedRestoring interface is not removed as prescribed in
the last steps then a future restore of a new savepoint will fail
because Flink will try to read legacy operator state that is not there
anymore.  The above steps also apply to Flink 1.3, when a user want's to
move away from the Checkpointed interface.


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

Branch: refs/heads/release-1.2
Commit: f68f6542bb2ba0171883d27c9ac75c74301ddf88
Parents: fdb3f65
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Apr 21 11:43:53 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:49:47 2017 +0200

----------------------------------------------------------------------
 .../streaming/api/operators/AbstractUdfStreamOperator.java   | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f68f6542/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
index 166287b..6a71a8a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
@@ -155,12 +155,14 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
 
 	@Override
 	public void restoreState(FSDataInputStream in) throws Exception {
+		boolean haveReadUdfStateFlag = false;
 		if (userFunction instanceof Checkpointed ||
-				(userFunction instanceof CheckpointedRestoring && in instanceof Migration)) {
+				(userFunction instanceof CheckpointedRestoring)) {
 			@SuppressWarnings("unchecked")
 			CheckpointedRestoring<Serializable> chkFunction = (CheckpointedRestoring<Serializable>) userFunction;
 
 			int hasUdfState = in.read();
+			haveReadUdfStateFlag = true;
 
 			if (hasUdfState == 1) {
 				Serializable functionState = InstantiationUtil.deserializeObject(in, getUserCodeClassloader());
@@ -172,7 +174,9 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
 					}
 				}
 			}
-		} else if (in instanceof Migration) {
+		}
+
+		if (in instanceof Migration && !haveReadUdfStateFlag) {
 			// absorb the introduced byte from the migration stream without too much further consequences
 			int hasUdfState = in.read();
 			if (hasUdfState == 1) {


[04/11] flink git commit: [FLINK-5969] Add OperatorSnapshotUtil

Posted by al...@apache.org.
[FLINK-5969] Add OperatorSnapshotUtil

This has methods for storing/reading OperatorStateHandles, as returned
from stream operator test harnesses. This can be used to write binary
snapshots for use in state migration tests.


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

Branch: refs/heads/release-1.2
Commit: 611434c6fa8e53cac25dd93f568d2670ec4ead72
Parents: 52fb578
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 12:31:53 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:04 2017 +0200

----------------------------------------------------------------------
 .../savepoint/SavepointV1Serializer.java        |  23 ++-
 .../streaming/util/OperatorSnapshotUtil.java    | 156 +++++++++++++++++++
 2 files changed, 172 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/611434c6/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
index ba1949a..a9fa3c6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.checkpoint.savepoint;
 
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.SubtaskState;
 import org.apache.flink.runtime.checkpoint.TaskState;
@@ -47,7 +49,8 @@ import java.util.Map;
  * that no default Java serialization is used for serialization. Therefore, we
  * don't rely on any involved Java classes to stay the same.
  */
-class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
+@Internal
+public class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 
 	private static final byte NULL_HANDLE = 0;
 	private static final byte BYTE_STREAM_STATE_HANDLE = 1;
@@ -209,7 +212,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 				keyedStateStream);
 	}
 
-	private static void serializeKeyGroupStateHandle(
+	@VisibleForTesting
+	public static void serializeKeyGroupStateHandle(
 			KeyGroupsStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle != null) {
@@ -225,7 +229,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 		}
 	}
 
-	private static KeyGroupsStateHandle deserializeKeyGroupStateHandle(DataInputStream dis) throws IOException {
+	@VisibleForTesting
+	public static KeyGroupsStateHandle deserializeKeyGroupStateHandle(DataInputStream dis) throws IOException {
 		final int type = dis.readByte();
 		if (NULL_HANDLE == type) {
 			return null;
@@ -245,7 +250,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 		}
 	}
 
-	private static void serializeOperatorStateHandle(
+	@VisibleForTesting
+	public static void serializeOperatorStateHandle(
 			OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle != null) {
@@ -273,7 +279,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 		}
 	}
 
-	private static OperatorStateHandle deserializeOperatorStateHandle(
+	@VisibleForTesting
+	public static OperatorStateHandle deserializeOperatorStateHandle(
 			DataInputStream dis) throws IOException {
 
 		final int type = dis.readByte();
@@ -304,7 +311,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 		}
 	}
 
-	private static void serializeStreamStateHandle(
+	@VisibleForTesting
+	public static void serializeStreamStateHandle(
 			StreamStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle == null) {
@@ -331,7 +339,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV1> {
 		dos.flush();
 	}
 
-	private static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException {
+	@VisibleForTesting
+	public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException {
 		int type = dis.read();
 		if (NULL_HANDLE == type) {
 			return null;

http://git-wip-us.apache.org/repos/asf/flink/blob/611434c6/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java
new file mode 100644
index 0000000..1a53598
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.util;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointV1Serializer;
+import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+
+/**
+ * Util for writing/reading {@link org.apache.flink.streaming.runtime.tasks.OperatorStateHandles},
+ * for use in tests.
+ */
+public class OperatorSnapshotUtil {
+
+	public static String getResourceFilename(String filename) {
+		ClassLoader cl = OperatorSnapshotUtil.class.getClassLoader();
+		URL resource = cl.getResource(filename);
+		return resource.getFile();
+	}
+
+	public static void writeStateHandle(OperatorStateHandles state, String path) throws IOException {
+		FileOutputStream out = new FileOutputStream(path);
+		DataOutputStream dos = new DataOutputStream(out);
+
+		dos.writeInt(state.getOperatorChainIndex());
+
+		SavepointV1Serializer.serializeStreamStateHandle(state.getLegacyOperatorState(), dos);
+
+		Collection<OperatorStateHandle> rawOperatorState = state.getRawOperatorState();
+		if (rawOperatorState != null) {
+			dos.writeInt(rawOperatorState.size());
+			for (OperatorStateHandle operatorStateHandle : rawOperatorState) {
+				SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos);
+			}
+		} else {
+			// this means no states, not even an empty list
+			dos.writeInt(-1);
+		}
+
+		Collection<OperatorStateHandle> managedOperatorState = state.getManagedOperatorState();
+		if (managedOperatorState != null) {
+			dos.writeInt(managedOperatorState.size());
+			for (OperatorStateHandle operatorStateHandle : managedOperatorState) {
+				SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos);
+			}
+		} else {
+			// this means no states, not even an empty list
+			dos.writeInt(-1);
+		}
+
+		Collection<KeyGroupsStateHandle> rawKeyedState = state.getRawKeyedState();
+		if (rawKeyedState != null) {
+			dos.writeInt(rawKeyedState.size());
+			for (KeyGroupsStateHandle keyedStateHandle : rawKeyedState) {
+				SavepointV1Serializer.serializeKeyGroupStateHandle(keyedStateHandle, dos);
+			}
+		} else {
+			// this means no operator states, not even an empty list
+			dos.writeInt(-1);
+		}
+
+		Collection<KeyGroupsStateHandle> managedKeyedState = state.getManagedKeyedState();
+		if (managedKeyedState != null) {
+			dos.writeInt(managedKeyedState.size());
+			for (KeyGroupsStateHandle keyedStateHandle : managedKeyedState) {
+				SavepointV1Serializer.serializeKeyGroupStateHandle(keyedStateHandle, dos);
+			}
+		} else {
+			// this means no operator states, not even an empty list
+			dos.writeInt(-1);
+		}
+
+		dos.flush();
+		out.close();
+	}
+
+	public static OperatorStateHandles readStateHandle(String path) throws IOException, ClassNotFoundException {
+		FileInputStream in = new FileInputStream(path);
+		DataInputStream dis = new DataInputStream(in);
+		int index = dis.readInt();
+
+		StreamStateHandle legacyState = SavepointV1Serializer.deserializeStreamStateHandle(dis);
+
+		List<OperatorStateHandle> rawOperatorState = null;
+		int numRawOperatorStates = dis.readInt();
+		if (numRawOperatorStates >= 0) {
+			rawOperatorState = new ArrayList<>();
+			for (int i = 0; i < numRawOperatorStates; i++) {
+				OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle(
+						dis);
+				rawOperatorState.add(operatorState);
+			}
+		}
+
+		List<OperatorStateHandle> managedOperatorState = null;
+		int numManagedOperatorStates = dis.readInt();
+		if (numManagedOperatorStates >= 0) {
+			managedOperatorState = new ArrayList<>();
+			for (int i = 0; i < numManagedOperatorStates; i++) {
+				OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle(
+						dis);
+				managedOperatorState.add(operatorState);
+			}
+		}
+
+		List<KeyGroupsStateHandle> rawKeyedState = null;
+		int numRawKeyedStates = dis.readInt();
+		if (numRawKeyedStates >= 0) {
+			rawKeyedState = new ArrayList<>();
+			for (int i = 0; i < numRawKeyedStates; i++) {
+				KeyGroupsStateHandle keyedState = SavepointV1Serializer.deserializeKeyGroupStateHandle(
+						dis);
+				rawKeyedState.add(keyedState);
+			}
+		}
+
+		List<KeyGroupsStateHandle> managedKeyedState = null;
+		int numManagedKeyedStates = dis.readInt();
+		if (numManagedKeyedStates >= 0) {
+			managedKeyedState = new ArrayList<>();
+			for (int i = 0; i < numManagedKeyedStates; i++) {
+				KeyGroupsStateHandle keyedState = SavepointV1Serializer.deserializeKeyGroupStateHandle(
+						dis);
+				managedKeyedState.add(keyedState);
+			}
+		}
+
+		return new OperatorStateHandles(index, legacyState, managedKeyedState, rawKeyedState, managedOperatorState, rawOperatorState);
+	}
+}


[11/11] flink git commit: [FLINK-5969] Add CEPFrom12MigrationTest

Posted by al...@apache.org.
[FLINK-5969] Add CEPFrom12MigrationTest

The binary snapshots have been created on the Flink 1.2 branch.


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

Branch: refs/heads/release-1.2
Commit: 852a710b4d91bdd319238c87d227c51a904070a7
Parents: 53432e0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Apr 28 12:28:50 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:05 2017 +0200

----------------------------------------------------------------------
 .../cep/operator/CEPFrom12MigrationTest.java    | 480 +++++++++++++++++++
 ...-migration-after-branching-flink1.2-snapshot | Bin 0 -> 5580 bytes
 ...-single-pattern-afterwards-flink1.2-snapshot | Bin 0 -> 2326 bytes
 ...ation-starting-new-pattern-flink1.2-snapshot | Bin 0 -> 5389 bytes
 4 files changed, 480 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/852a710b/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
new file mode 100644
index 0000000..9a15754
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java
@@ -0,0 +1,480 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.functions.FilterFunction;
+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.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.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, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new NFAFactory()),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		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");
+
+		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, Event>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								keySelector,
+								IntSerializer.INSTANCE,
+								new NFAFactory()),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		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, Event> patternMap1 = (Map<String, Event>) resultRecord1.getValue();
+
+		assertEquals(startEvent, patternMap1.get("start"));
+		assertEquals(middleEvent1, patternMap1.get("middle"));
+		assertEquals(endEvent, patternMap1.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap2 = (Map<String, Event>) resultRecord2.getValue();
+
+		assertEquals(startEvent, patternMap2.get("start"));
+		assertEquals(middleEvent2, patternMap2.get("middle"));
+		assertEquals(endEvent, patternMap2.get("end"));
+
+		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, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new NFAFactory()),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		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");
+
+		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, Event>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								keySelector,
+								IntSerializer.INSTANCE,
+								new NFAFactory()),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		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, Event> patternMap1 = (Map<String, Event>) resultRecord1.getValue();
+
+		assertEquals(startEvent1, patternMap1.get("start"));
+		assertEquals(middleEvent1, patternMap1.get("middle"));
+		assertEquals(endEvent, patternMap1.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap2 = (Map<String, Event>) resultRecord2.getValue();
+
+		assertEquals(startEvent1, patternMap2.get("start"));
+		assertEquals(middleEvent2, patternMap2.get("middle"));
+		assertEquals(endEvent, patternMap2.get("end"));
+
+		@SuppressWarnings("unchecked")
+		Map<String, Event> patternMap3 = (Map<String, Event>) resultRecord3.getValue();
+
+		assertEquals(startEvent2, patternMap3.get("start"));
+		assertEquals(middleEvent2, patternMap3.get("middle"));
+		assertEquals(endEvent, patternMap3.get("end"));
+
+		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, Event>> harness =
+			new KeyedOneInputStreamOperatorTestHarness<>(
+				new KeyedCEPPatternOperator<>(
+					Event.createTypeSerializer(),
+					false,
+					keySelector,
+					IntSerializer.INSTANCE,
+					new SinglePatternNFAFactory()),
+				keySelector,
+				BasicTypeInfo.INT_TYPE_INFO);
+
+		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");
+
+		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, Event>> harness =
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						new KeyedCEPPatternOperator<>(
+								Event.createTypeSerializer(),
+								false,
+								keySelector,
+								IntSerializer.INSTANCE,
+								new SinglePatternNFAFactory()),
+						keySelector,
+						BasicTypeInfo.INT_TYPE_INFO);
+
+		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, Event> patternMap = (Map<String, Event>) resultRecord.getValue();
+
+		assertEquals(startEvent1, patternMap.get("start"));
+
+		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 implements FilterFunction<Event> {
+		private static final long serialVersionUID = 5726188262756267490L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("start");
+		}
+	}
+
+	private static class MiddleFilter implements FilterFunction<SubEvent> {
+		private static final long serialVersionUID = 6215754202506583964L;
+
+		@Override
+		public boolean filter(SubEvent value) throws Exception {
+			return value.getVolume() > 5.0;
+		}
+	}
+
+	private static class EndFilter implements FilterFunction<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/852a710b/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
new file mode 100644
index 0000000..6775f2a
Binary files /dev/null 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/852a710b/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
new file mode 100644
index 0000000..f63b7dd
Binary files /dev/null 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/852a710b/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
new file mode 100644
index 0000000..8e0fd27
Binary files /dev/null and b/flink-libraries/flink-cep/src/test/resources/cep-migration-starting-new-pattern-flink1.2-snapshot differ


[08/11] flink git commit: [FLINK-5969] Add WindowOperatorFrom12MigrationTest

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/c9ba4f03/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
deleted file mode 100644
index 7a356cf..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java
+++ /dev/null
@@ -1,888 +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 WindowOperatorMigrationTest {
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = WindowOperatorMigrationTest.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);
-
-		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);
-
-		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);
-
-		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);
-
-		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);
-
-		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);
-
-		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,
-				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,
-				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().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/c9ba4f03/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.2-snapshot b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.2-snapshot
new file mode 100644
index 0000000..d182bee
Binary files /dev/null and b/flink-streaming-java/src/test/resources/win-op-migration-test-accum-aligned-flink1.2-snapshot differ

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

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

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

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

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

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

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


[03/11] flink git commit: [FLINK-5969] Add ContinuousFileProcessingFrom12MigrationTest

Posted by al...@apache.org.
[FLINK-5969] Add ContinuousFileProcessingFrom12MigrationTest

The binary snapshots were created on the Flink 1.2 branch.


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

Branch: refs/heads/release-1.2
Commit: 62601b4978582f05e5fadec83ce18eeec5842686
Parents: 611434c
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 11:25:32 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:04 2017 +0200

----------------------------------------------------------------------
 ...inuousFileProcessingFrom11MigrationTest.java | 401 +++++++++++++++++++
 ...inuousFileProcessingFrom12MigrationTest.java | 361 +++++++++++++++++
 .../ContinuousFileProcessingMigrationTest.java  | 401 -------------------
 ...gration-test-1493116191000-flink1.2-snapshot | Bin 0 -> 373 bytes
 .../reader-migration-test-flink1.2-snapshot     | Bin 0 -> 2590 bytes
 5 files changed, 762 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/62601b49/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
new file mode 100644
index 0000000..5ff0450
--- /dev/null
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom11MigrationTest.java
@@ -0,0 +1,401 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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());
+		}
+		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/62601b49/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
new file mode 100644
index 0000000..6ecd401
--- /dev/null
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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();
+						}
+					});
+				}
+				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/62601b49/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
deleted file mode 100644
index 440bfcc..0000000
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
+++ /dev/null
@@ -1,401 +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.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 ContinuousFileProcessingMigrationTest {
-
-	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 = ContinuousFileProcessingMigrationTest.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());
-		}
-		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/62601b49/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1493116191000-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1493116191000-flink1.2-snapshot b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1493116191000-flink1.2-snapshot
new file mode 100644
index 0000000..25451ac
Binary files /dev/null and b/flink-fs-tests/src/test/resources/monitoring-function-migration-test-1493116191000-flink1.2-snapshot differ

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


[10/11] flink git commit: [FLINK-5969] Add BucketingSinkFrom12MigrationTest

Posted by al...@apache.org.
[FLINK-5969] Add BucketingSinkFrom12MigrationTest

The binary snapshots have been created on the Flink 1.2 branch.


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

Branch: refs/heads/release-1.2
Commit: 53432e0690510aa5838165d4958777d872668be4
Parents: c9ba4f0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 17:50:59 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:05 2017 +0200

----------------------------------------------------------------------
 .../connectors/fs/bucketing/BucketingSink.java  |   6 +
 .../BucketingSinkFrom12MigrationTest.java       | 223 +++++++++++++++++++
 ...keting-sink-migration-test-flink1.2-snapshot | Bin 0 -> 1623 bytes
 3 files changed, 229 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/53432e06/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
index 7dbcda7..20e54b8 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.connectors.fs.bucketing;
 
 import org.apache.commons.lang3.time.StopWatch;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.OperatorStateStore;
@@ -1033,6 +1034,11 @@ public class BucketingSink<T>
 		return this;
 	}
 
+	@VisibleForTesting
+	public State<T> getState() {
+		return state;
+	}
+
 	// --------------------------------------------------------------------------------------------
 	//  Internal Classes
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/53432e06/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
new file mode 100644
index 0000000..350b7b4
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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/53432e06/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot b/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot
new file mode 100644
index 0000000..a541bad
Binary files /dev/null and b/flink-connectors/flink-connector-filesystem/src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot differ


[02/11] flink git commit: [FLINK-5969] Augment SavepointMigrationTestBase to catch failed jobs

Posted by al...@apache.org.
[FLINK-5969] Augment SavepointMigrationTestBase to catch failed jobs


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

Branch: refs/heads/release-1.2
Commit: a3ccffcb0a1b3e936fff31a61dddf1cada8ab99b
Parents: f68f654
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Apr 20 14:48:22 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:03 2017 +0200

----------------------------------------------------------------------
 .../flink/client/program/ClusterClient.java     | 32 ++++++++++++++
 .../utils/SavepointMigrationTestBase.java       | 44 +++++++++++++++++---
 2 files changed, 70 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a3ccffcb/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
index 8d0e841..ab4daa9 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java
@@ -489,6 +489,38 @@ public abstract class ClusterClient {
 	}
 
 	/**
+	 * Reattaches to a running job with the given job id.
+	 *
+	 * @param jobID The job id of the job to attach to
+	 * @return The JobExecutionResult for the jobID
+	 * @throws JobExecutionException if an error occurs during monitoring the job execution
+	 */
+	public JobListeningContext connectToJob(JobID jobID) throws JobExecutionException {
+		final LeaderRetrievalService leaderRetrievalService;
+		try {
+			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
+		} catch (Exception e) {
+			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
+		}
+
+		ActorGateway jobManagerGateway;
+		try {
+			jobManagerGateway = getJobManagerGateway();
+		} catch (Exception e) {
+			throw new JobRetrievalException(jobID, "Could not retrieve the JobManager Gateway", e);
+		}
+
+		return JobClient.attachToRunningJob(
+				jobID,
+				jobManagerGateway,
+				flinkConfig,
+				actorSystemLoader.get(),
+				leaderRetrievalService,
+				timeout,
+				printStatusDuringExecution);
+	}
+
+	/**
 	 * Cancels a job identified by the job id.
 	 * @param jobId the job id
 	 * @throws Exception In case an error occurred.

http://git-wip-us.apache.org/repos/asf/flink/blob/a3ccffcb/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 2395a4b..ed69858 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.test.checkpointing.utils;
 
+import akka.pattern.Patterns;
+import akka.util.Timeout;
 import java.io.File;
 import java.net.URI;
 import java.net.URL;
@@ -25,13 +27,16 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
@@ -48,6 +53,7 @@ import scala.Option;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.Duration;
 import scala.concurrent.duration.FiniteDuration;
 
 import static junit.framework.Assert.fail;
@@ -103,7 +109,8 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
 	}
 
-	protected void executeAndSavepoint(
+	@SafeVarargs
+	protected final void executeAndSavepoint(
 			StreamExecutionEnvironment env,
 			String savepointPath,
 			Tuple2<String, Integer>... expectedAccumulators) throws Exception {
@@ -194,11 +201,8 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 			String savepointPath,
 			Tuple2<String, Integer>... expectedAccumulators) throws Exception {
 
-		int parallelism = env.getParallelism();
-
 		// Retrieve the job manager
-
-		ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft());
+		Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft());
 
 		// Submit the job
 		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
@@ -208,11 +212,38 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph);
 
 		StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration());
+		JobListeningContext jobListeningContext = clusterClient.connectToJob(jobSubmissionResult.getJobID());
 
 		boolean done = false;
 		while (DEADLINE.hasTimeLeft()) {
+
+			// try and get a job result, this will fail if the job already failed. Use this
+			// to get out of this loop
+			JobID jobId = jobSubmissionResult.getJobID();
+			FiniteDuration timeout = FiniteDuration.apply(5, TimeUnit.SECONDS);
+
+			try {
+
+				Future<Object> future = clusterClient
+						.getJobManagerGateway()
+						.ask(JobManagerMessages.getRequestJobStatus(jobSubmissionResult.getJobID()), timeout);
+
+				Object result = Await.result(future, timeout);
+
+				if (result instanceof JobManagerMessages.CurrentJobStatus) {
+					if (((JobManagerMessages.CurrentJobStatus) result).status() == JobStatus.FAILED) {
+						Object jobResult = Await.result(
+								jobListeningContext.getJobResultFuture(),
+								Duration.apply(5, TimeUnit.SECONDS));
+						fail("Job failed: " + jobResult);
+					}
+				}
+			} catch (Exception e) {
+				fail("Could not connect to job: " + e);
+			}
+
 			Thread.sleep(100);
-			Map<String, Object> accumulators = clusterClient.getAccumulators(jobSubmissionResult.getJobID());
+			Map<String, Object> accumulators = clusterClient.getAccumulators(jobId);
 
 			boolean allDone = true;
 			for (Tuple2<String, Integer> acc : expectedAccumulators) {
@@ -226,6 +257,7 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 					break;
 				}
 			}
+
 			if (allDone) {
 				done = true;
 				break;


[09/11] flink git commit: [FLINK-5969] Add WindowOperatorFrom12MigrationTest

Posted by al...@apache.org.
[FLINK-5969] Add WindowOperatorFrom12MigrationTest

The binary snapshots for this were created on the Flink 1.2 branch.


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

Branch: refs/heads/release-1.2
Commit: c9ba4f0313346d96db6258b533041da180f5471e
Parents: c89d4b4
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 17:13:27 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:05 2017 +0200

----------------------------------------------------------------------
 .../WindowOperatorFrom11MigrationTest.java      |  888 ++++++++++++++++
 .../WindowOperatorFrom12MigrationTest.java      | 1000 ++++++++++++++++++
 .../windowing/WindowOperatorMigrationTest.java  |  888 ----------------
 ...gration-test-accum-aligned-flink1.2-snapshot |  Bin 0 -> 222 bytes
 ...igration-test-aggr-aligned-flink1.2-snapshot |  Bin 0 -> 187 bytes
 ...tion-test-apply-event-time-flink1.2-snapshot |  Bin 0 -> 2193 bytes
 ...test-apply-processing-time-flink1.2-snapshot |  Bin 0 -> 2081 bytes
 ...ion-test-reduce-event-time-flink1.2-snapshot |  Bin 0 -> 1988 bytes
 ...est-reduce-processing-time-flink1.2-snapshot |  Bin 0 -> 1925 bytes
 ...sion-with-stateful-trigger-flink1.2-snapshot |  Bin 0 -> 3703 bytes
 ...with-stateful-trigger-mint-flink1.2-snapshot |  Bin 0 -> 494 bytes
 11 files changed, 1888 insertions(+), 888 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c9ba4f03/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
new file mode 100644
index 0000000..3e8d49d
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom11MigrationTest.java
@@ -0,0 +1,888 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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,
+				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,
+				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().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/c9ba4f03/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
new file mode 100644
index 0000000..d3e4414
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorFrom12MigrationTest.java
@@ -0,0 +1,1000 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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);
+
+		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,
+				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 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,
+				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()));
+		}
+	}
+}


[05/11] flink git commit: [FLINK-5969] Add savepoint IT case that checks restore from 1.2

Posted by al...@apache.org.
[FLINK-5969] Add savepoint IT case that checks restore from 1.2

The binary savepoints in this were created on the Flink 1.2.0 release
commit.


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

Branch: refs/heads/release-1.2
Commit: 52fb578953fb571f5787c8f69fe2ac4525488a3d
Parents: a3ccffc
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Apr 20 17:09:00 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:04 2017 +0200

----------------------------------------------------------------------
 .../utils/SavepointMigrationTestBase.java       |  24 +-
 ...atefulJobSavepointFrom11MigrationITCase.java | 562 ++++++++++++++
 ...atefulJobSavepointFrom12MigrationITCase.java | 769 +++++++++++++++++++
 .../StatefulUDFSavepointMigrationITCase.java    | 562 --------------
 ...eful-udf-migration-itcase-flink1.2-savepoint | Bin 0 -> 25245 bytes
 ...-migration-itcase-flink1.2-savepoint-rocksdb | Bin 0 -> 25256 bytes
 6 files changed, 1332 insertions(+), 585 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/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 ed69858..f5ef2b2 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
@@ -156,15 +156,10 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		}
 
 		LOG.info("Triggering savepoint.");
-		// Flink 1.2
+
 		final Future<Object> savepointResultFuture =
 				jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobSubmissionResult.getJobID(), Option.<String>empty()), DEADLINE.timeLeft());
 
-		// Flink 1.1
-//		final Future<Object> savepointResultFuture =
-//				jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobSubmissionResult.getJobID()), DEADLINE.timeLeft());
-
-
 		Object savepointResult = Await.result(savepointResultFuture, DEADLINE.timeLeft());
 
 		if (savepointResult instanceof JobManagerMessages.TriggerSavepointFailure) {
@@ -175,24 +170,7 @@ public class SavepointMigrationTestBase extends TestBaseUtils {
 		final String jobmanagerSavepointPath = ((JobManagerMessages.TriggerSavepointSuccess) savepointResult).savepointPath();
 		LOG.info("Saved savepoint: " + jobmanagerSavepointPath);
 
-		// Flink 1.2
 		FileUtils.moveFile(new File(new URI(jobmanagerSavepointPath).getPath()), new File(savepointPath));
-
-		// Flink 1.1
-		// Retrieve the savepoint from the testing job manager
-//		LOG.info("Requesting the savepoint.");
-//		Future<Object> savepointFuture = jobManager.ask(new TestingJobManagerMessages.RequestSavepoint(jobmanagerSavepointPath), DEADLINE.timeLeft());
-//
-//		Savepoint savepoint = ((TestingJobManagerMessages.ResponseSavepoint) Await.result(savepointFuture, DEADLINE.timeLeft())).savepoint();
-//		LOG.info("Retrieved savepoint: " + jobmanagerSavepointPath + ".");
-//
-//		LOG.info("Storing savepoint to file.");
-//		Configuration config = new Configuration();
-//		config.setString(org.apache.flink.runtime.checkpoint.savepoint.SavepointStoreFactory.SAVEPOINT_BACKEND_KEY, "filesystem");
-//		config.setString(org.apache.flink.runtime.checkpoint.savepoint.SavepointStoreFactory.SAVEPOINT_DIRECTORY_KEY, "file:///Users/aljoscha/Downloads");
-//		String path = org.apache.flink.runtime.checkpoint.savepoint.SavepointStoreFactory.createFromConfig(config).storeSavepoint(savepoint);
-//
-//		FileUtils.moveFile(new File(new URI(path).getPath()), new File(savepointPath));
 	}
 
 	@SafeVarargs

http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
new file mode 100644
index 0000000..4d94d25
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.java
@@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * This verifies that we can restore a complete job from a Flink 1.1 savepoint.
+ *
+ * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
+ */
+public class StatefulJobSavepointFrom11MigrationITCase extends SavepointMigrationTestBase {
+	private static final int NUM_SOURCE_ELEMENTS = 4;
+	private static final String EXPECTED_ELEMENTS_ACCUMULATOR = "NUM_EXPECTED_ELEMENTS";
+	private static final String SUCCESSFUL_CHECK_ACCUMULATOR = "SUCCESSFUL_CHECKS";
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.1.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink11() throws Exception {
+
+		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);
+		env.setMaxParallelism(4);
+
+		// create source
+		env
+				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
+
+		executeAndSavepoint(
+				env,
+				"src/test/resources/stateful-udf-migration-itcase-flink1.1-savepoint",
+				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.1.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink11WithRocksDB() throws Exception {
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		RocksDBStateBackend rocksBackend =
+				new RocksDBStateBackend(new MemoryStateBackend());
+//		rocksBackend.enableFullyAsyncSnapshots();
+		env.setStateBackend(rocksBackend);
+		env.enableCheckpointing(500);
+		env.setParallelism(4);
+		env.setMaxParallelism(4);
+
+		// create source
+		env
+				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
+
+		executeAndSavepoint(
+				env,
+				"src/test/resources/stateful-udf-migration-itcase-flink1.1-savepoint-rocksdb",
+				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+
+	@Test
+	public void testSavepointRestoreFromFlink11() throws Exception {
+
+		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
+
+		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);
+		env.setMaxParallelism(4);
+
+		// create source
+		env
+				.addSource(new RestoringCheckingSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new RestoringCheckingFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new RestoringCheckingFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateCheckingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
+
+		restoreAndExecute(
+				env,
+				getResourceFilename("stateful-udf-migration-itcase-flink1.1-savepoint"),
+				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
+	}
+
+	@Test
+	public void testSavepointRestoreFromFlink11FromRocksDB() throws Exception {
+
+		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		// we only test memory state backend yet
+		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
+		env.enableCheckpointing(500);
+		env.setParallelism(4);
+		env.setMaxParallelism(4);
+
+		// create source
+		env
+				.addSource(new RestoringCheckingSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new RestoringCheckingFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new RestoringCheckingFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateCheckingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
+
+		restoreAndExecute(
+				env,
+				getResourceFilename("stateful-udf-migration-itcase-flink1.1-savepoint-rocksdb"),
+				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
+	}
+
+	private static class LegacyCheckpointedSource
+			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
+
+		public static String CHECKPOINTED_STRING = "Here be dragons!";
+
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean isRunning = true;
+
+		private final int numElements;
+
+		public LegacyCheckpointedSource(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
+
+			synchronized (ctx.getCheckpointLock()) {
+				for (long i = 0; i < numElements; i++) {
+					ctx.collect(new Tuple2<>(i, i));
+				}
+			}
+			while (isRunning) {
+				Thread.sleep(20);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void restoreState(String state) throws Exception {
+			assertEquals(CHECKPOINTED_STRING, state);
+		}
+
+		@Override
+		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_STRING;
+		}
+	}
+
+	private static class RestoringCheckingSource
+			extends RichSourceFunction<Tuple2<Long, Long>>
+			implements CheckpointedRestoring<String> {
+
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean isRunning = true;
+
+		private final int numElements;
+
+		private String restoredState;
+
+		public RestoringCheckingSource(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
+			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
+
+			synchronized (ctx.getCheckpointLock()) {
+				for (long i = 0; i < numElements; i++) {
+					ctx.collect(new Tuple2<>(i, i));
+				}
+			}
+
+			while (isRunning) {
+				Thread.sleep(20);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void restoreState(String state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements Checkpointed<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+				new Tuple2<>("hello", 42L);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+		}
+
+		@Override
+		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_TUPLE;
+		}
+	}
+
+	public static class RestoringCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements CheckpointedRestoring<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private transient Tuple2<String, Long> restoredState;
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
+
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class LegacyCheckpointedFlatMapWithKeyedState
+			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements Checkpointed<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+				new Tuple2<>("hello", 42L);
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			getRuntimeContext().getState(stateDescriptor).update(value.f1);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+		}
+
+		@Override
+		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_TUPLE;
+		}
+	}
+
+	public static class RestoringCheckingFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements CheckpointedRestoring<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private transient Tuple2<String, Long> restoredState;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
+			if (state == null) {
+				throw new RuntimeException("Missing key value state for " + value);
+			}
+
+			assertEquals(value.f1, state.value());
+			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			getRuntimeContext().getState(stateDescriptor).update(value.f1);
+		}
+	}
+
+	public static class KeyedStateCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
+			if (state == null) {
+				throw new RuntimeException("Missing key value state for " + value);
+			}
+
+			assertEquals(value.f1, state.value());
+			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
+		}
+	}
+
+	public static class CheckpointedUdfOperator
+			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		private static final String CHECKPOINTED_STRING = "Oh my, that's nice!";
+
+		public CheckpointedUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			output.collect(element);
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			output.emitWatermark(mark);
+		}
+
+		// Flink 1.1
+//		@Override
+//		public StreamTaskState snapshotOperatorState(
+//				long checkpointId, long timestamp) throws Exception {
+//			StreamTaskState result = super.snapshotOperatorState(checkpointId, timestamp);
+//
+//			AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(
+//					checkpointId,
+//					timestamp);
+//
+//			out.writeUTF(CHECKPOINTED_STRING);
+//
+//			result.setOperatorState(out.closeAndGetHandle());
+//
+//			return result;
+//		}
+	}
+
+	public static class RestoringCheckingUdfOperator
+			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		private String restoredState;
+
+		public RestoringCheckingUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void open() throws Exception {
+			super.open();
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
+
+			assertEquals(CheckpointedUdfOperator.CHECKPOINTED_STRING, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			output.emitWatermark(mark);
+		}
+
+		@Override
+		public void restoreState(FSDataInputStream in) throws Exception {
+			super.restoreState(in);
+
+			DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
+
+			restoredState = streamWrapper.readUTF();
+		}
+	}
+
+	public static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		private final String accumulatorName;
+
+		int count = 0;
+
+		public AccumulatorCountingSink(String accumulatorName) {
+			this.accumulatorName = accumulatorName;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(accumulatorName, new IntCounter());
+		}
+
+		@Override
+		public void invoke(T value) throws Exception {
+			count++;
+			getRuntimeContext().getAccumulator(accumulatorName).add(1);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/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
new file mode 100644
index 0000000..e60cb5d
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
@@ -0,0 +1,769 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 static org.junit.Assert.assertEquals;
+
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
+ *
+ * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
+ *
+ * <p>The tests will time out if they don't see the required number of successful checks within
+ * a time limit.
+ */
+public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
+	private static final int NUM_SOURCE_ELEMENTS = 4;
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.2.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink12() throws Exception {
+
+		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);
+		env.setMaxParallelism(4);
+
+		env
+				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.keyBy(0)
+				.transform(
+						"timely_stateful_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
+
+		executeAndSavepoint(
+				env,
+				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
+				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+	/**
+	 * This has to be manually executed to create the savepoint on Flink 1.2.
+	 */
+	@Test
+	@Ignore
+	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		RocksDBStateBackend rocksBackend =
+				new RocksDBStateBackend(new MemoryStateBackend());
+		env.setStateBackend(rocksBackend);
+		env.enableCheckpointing(500);
+		env.setParallelism(4);
+		env.setMaxParallelism(4);
+
+		env
+				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
+				.keyBy(0)
+				.transform(
+						"timely_stateful_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
+
+		executeAndSavepoint(
+				env,
+				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
+				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+
+	@Test
+	public void testSavepointRestoreFromFlink12() throws Exception {
+
+		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);
+		env.setMaxParallelism(4);
+
+		env
+				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
+				.keyBy(0)
+				.transform(
+						"timely_stateful_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
+
+		restoreAndExecute(
+				env,
+				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
+				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),
+				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+	@Test
+	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
+
+		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);
+		env.setMaxParallelism(4);
+
+		env
+				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
+				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
+				.keyBy(0)
+				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
+				.keyBy(0)
+				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
+				.keyBy(0)
+				.transform(
+						"custom_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
+				.keyBy(0)
+				.transform(
+						"timely_stateful_operator",
+						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
+						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
+				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
+
+		restoreAndExecute(
+				env,
+				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
+				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),
+				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
+				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
+	}
+
+	private static class LegacyCheckpointedSource
+			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
+
+		public static String CHECKPOINTED_STRING = "Here be dragons!";
+
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean isRunning = true;
+
+		private final int numElements;
+
+		public LegacyCheckpointedSource(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
+
+			ctx.emitWatermark(new Watermark(0));
+
+			synchronized (ctx.getCheckpointLock()) {
+				for (long i = 0; i < numElements; i++) {
+					ctx.collect(new Tuple2<>(i, i));
+				}
+			}
+
+			// don't emit a final watermark so that we don't trigger the registered event-time
+			// timers
+			while (isRunning) {
+				Thread.sleep(20);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void restoreState(String state) throws Exception {
+			assertEquals(CHECKPOINTED_STRING, state);
+		}
+
+		@Override
+		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_STRING;
+		}
+	}
+
+	private static class CheckingRestoringSource
+			extends RichSourceFunction<Tuple2<Long, Long>>
+			implements CheckpointedRestoring<String> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
+
+		private volatile boolean isRunning = true;
+
+		private final int numElements;
+
+		private String restoredState;
+
+		public CheckingRestoringSource(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
+			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+
+			// immediately trigger any set timers
+			ctx.emitWatermark(new Watermark(1000));
+
+			synchronized (ctx.getCheckpointLock()) {
+				for (long i = 0; i < numElements; i++) {
+					ctx.collect(new Tuple2<>(i, i));
+				}
+			}
+
+			while (isRunning) {
+				Thread.sleep(20);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		@Override
+		public void restoreState(String state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements Checkpointed<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+				new Tuple2<>("hello", 42L);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+		}
+
+		@Override
+		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_TUPLE;
+		}
+	}
+
+	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements CheckpointedRestoring<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
+
+		private transient Tuple2<String, Long> restoredState;
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class LegacyCheckpointedFlatMapWithKeyedState
+			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements Checkpointed<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
+				new Tuple2<>("hello", 42L);
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			getRuntimeContext().getState(stateDescriptor).update(value.f1);
+
+			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+		}
+
+		@Override
+		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+			return CHECKPOINTED_TUPLE;
+		}
+	}
+
+	public static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements CheckpointedRestoring<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedState.class + "_RESTORE_CHECK";
+
+		private transient Tuple2<String, Long> restoredState;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
+			if (state == null) {
+				throw new RuntimeException("Missing key value state for " + value);
+			}
+
+			assertEquals(value.f1, state.value());
+			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
+			implements CheckpointedRestoring<Tuple2<String, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedStateInOperator.class + "_RESTORE_CHECK";
+
+		private transient Tuple2<String, Long> restoredState;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
+			if (state == null) {
+				throw new RuntimeException("Missing key value state for " + value);
+			}
+
+			assertEquals(value.f1, state.value());
+			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void restoreState(Tuple2<String, Long> state) throws Exception {
+			restoredState = state;
+		}
+	}
+
+	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			getRuntimeContext().getState(stateDescriptor).update(value.f1);
+		}
+	}
+
+	public static class CheckingKeyedStateFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingKeyedStateFlatMap.class + "_RESTORE_CHECK";
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
+			out.collect(value);
+
+			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
+			if (state == null) {
+				throw new RuntimeException("Missing key value state for " + value);
+			}
+
+			assertEquals(value.f1, state.value());
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+		}
+	}
+
+	public static class CheckpointedUdfOperator
+			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		private static final String CHECKPOINTED_STRING = "Oh my, that's nice!";
+
+		public CheckpointedUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			output.emitWatermark(mark);
+		}
+
+		@Override
+		public void snapshotState(
+				FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
+			super.snapshotState(out, checkpointId, timestamp);
+
+			DataOutputViewStreamWrapper streamWrapper = new DataOutputViewStreamWrapper(out);
+
+			streamWrapper.writeUTF(CHECKPOINTED_STRING);
+			streamWrapper.flush();
+		}
+	}
+
+	public static class CheckingRestoringUdfOperator
+			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
+
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringUdfOperator.class + "_RESTORE_CHECK";
+
+		private String restoredState;
+
+		public CheckingRestoringUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void open() throws Exception {
+			super.open();
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
+
+			assertEquals(CheckpointedUdfOperator.CHECKPOINTED_STRING, restoredState);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			output.emitWatermark(mark);
+		}
+
+		@Override
+		public void restoreState(FSDataInputStream in) throws Exception {
+			super.restoreState(in);
+
+			DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
+
+			restoredState = streamWrapper.readUTF();
+		}
+	}
+
+	public static class TimelyStatefulOperator
+			extends AbstractStreamOperator<Tuple2<Long, Long>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
+		private static final long serialVersionUID = 1L;
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		private transient InternalTimerService<Long> timerService;
+
+		@Override
+		public void open() throws Exception {
+			super.open();
+
+			timerService = getInternalTimerService(
+					"timer",
+					LongSerializer.INSTANCE,
+					this);
+
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
+					element.getValue().f0,
+					LongSerializer.INSTANCE,
+					stateDescriptor);
+
+			state.update(element.getValue().f1);
+
+			timerService.registerEventTimeTimer(element.getValue().f0, timerService.currentWatermark() + 10);
+			timerService.registerProcessingTimeTimer(element.getValue().f0, timerService.currentProcessingTime() + 30_000);
+
+			output.collect(element);
+		}
+
+		@Override
+		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
+
+		}
+
+		@Override
+		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
+
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			output.emitWatermark(mark);
+		}
+	}
+
+	public static class CheckingTimelyStatefulOperator
+			extends AbstractStreamOperator<Tuple2<Long, Long>>
+			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
+		private static final long serialVersionUID = 1L;
+
+		public static final String SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PROCESS_CHECKS";
+		public static final String SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_ET_CHECKS";
+		public static final String SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PT_CHECKS";
+
+		private final ValueStateDescriptor<Long> stateDescriptor =
+				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
+
+		private transient InternalTimerService<Long> timerService;
+
+		@Override
+		public void open() throws Exception {
+			super.open();
+
+			timerService = getInternalTimerService(
+					"timer",
+					LongSerializer.INSTANCE,
+					this);
+
+			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, new IntCounter());
+			getRuntimeContext().addAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, new IntCounter());
+			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
+			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
+					element.getValue().f0,
+					LongSerializer.INSTANCE,
+					stateDescriptor);
+
+			assertEquals(state.value(), element.getValue().f1);
+			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR).add(1);
+
+			output.collect(element);
+		}
+
+		@Override
+		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
+			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
+					timer.getNamespace(),
+					LongSerializer.INSTANCE,
+					stateDescriptor);
+
+			assertEquals(state.value(), timer.getNamespace());
+			getRuntimeContext().getAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR).add(1);
+		}
+
+		@Override
+		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
+			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
+					timer.getNamespace(),
+					LongSerializer.INSTANCE,
+					stateDescriptor);
+
+			assertEquals(state.value(), timer.getNamespace());
+			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR).add(1);
+		}
+	}
+
+	public static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
+		private static final long serialVersionUID = 1L;
+
+		public static final String NUM_ELEMENTS_ACCUMULATOR = AccumulatorCountingSink.class + "_NUM_ELEMENTS";
+
+		int count = 0;
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+
+			getRuntimeContext().addAccumulator(NUM_ELEMENTS_ACCUMULATOR, new IntCounter());
+		}
+
+		@Override
+		public void invoke(T value) throws Exception {
+			count++;
+			getRuntimeContext().getAccumulator(NUM_ELEMENTS_ACCUMULATOR).add(1);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulUDFSavepointMigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulUDFSavepointMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulUDFSavepointMigrationITCase.java
deleted file mode 100644
index 10a8998..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulUDFSavepointMigrationITCase.java
+++ /dev/null
@@ -1,562 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.checkpointing.utils;
-
-import org.apache.flink.api.common.accumulators.IntCounter;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * This verifies that we can restore a complete job from a Flink 1.1 savepoint.
- *
- * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
- */
-public class StatefulUDFSavepointMigrationITCase extends SavepointMigrationTestBase {
-	private static final int NUM_SOURCE_ELEMENTS = 4;
-	private static final String EXPECTED_ELEMENTS_ACCUMULATOR = "NUM_EXPECTED_ELEMENTS";
-	private static final String SUCCESSFUL_CHECK_ACCUMULATOR = "SUCCESSFUL_CHECKS";
-
-	/**
-	 * This has to be manually executed to create the savepoint on Flink 1.1.
-	 */
-	@Test
-	@Ignore
-	public void testCreateSavepointOnFlink11() throws Exception {
-
-		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);
-		env.setMaxParallelism(4);
-
-		// create source
-		env
-				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
-				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
-				.keyBy(0)
-				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
-				.keyBy(0)
-				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
-				.keyBy(0)
-				.transform(
-						"custom_operator",
-						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
-						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
-				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
-
-		executeAndSavepoint(
-				env,
-				"src/test/resources/stateful-udf-migration-itcase-flink1.1-savepoint",
-				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
-	}
-
-	/**
-	 * This has to be manually executed to create the savepoint on Flink 1.1.
-	 */
-	@Test
-	@Ignore
-	public void testCreateSavepointOnFlink11WithRocksDB() throws Exception {
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		RocksDBStateBackend rocksBackend =
-				new RocksDBStateBackend(new MemoryStateBackend());
-//		rocksBackend.enableFullyAsyncSnapshots();
-		env.setStateBackend(rocksBackend);
-		env.enableCheckpointing(500);
-		env.setParallelism(4);
-		env.setMaxParallelism(4);
-
-		// create source
-		env
-				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
-				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
-				.keyBy(0)
-				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
-				.keyBy(0)
-				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
-				.keyBy(0)
-				.transform(
-						"custom_operator",
-						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
-						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
-				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
-
-		executeAndSavepoint(
-				env,
-				"src/test/resources/stateful-udf-migration-itcase-flink1.1-savepoint-rocksdb",
-				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
-	}
-
-
-	@Test
-	public void testSavepointRestoreFromFlink11() throws Exception {
-
-		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
-
-		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);
-		env.setMaxParallelism(4);
-
-		// create source
-		env
-				.addSource(new RestoringCheckingSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
-				.flatMap(new RestoringCheckingFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
-				.keyBy(0)
-				.flatMap(new RestoringCheckingFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
-				.keyBy(0)
-				.flatMap(new KeyedStateCheckingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
-				.keyBy(0)
-				.transform(
-						"custom_operator",
-						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
-						new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
-				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
-
-		restoreAndExecute(
-				env,
-				getResourceFilename("stateful-udf-migration-itcase-flink1.1-savepoint"),
-				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
-	}
-
-	@Test
-	public void testSavepointRestoreFromFlink11FromRocksDB() throws Exception {
-
-		final int EXPECTED_SUCCESSFUL_CHECKS = 21;
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		// we only test memory state backend yet
-		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
-		env.enableCheckpointing(500);
-		env.setParallelism(4);
-		env.setMaxParallelism(4);
-
-		// create source
-		env
-				.addSource(new RestoringCheckingSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
-				.flatMap(new RestoringCheckingFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
-				.keyBy(0)
-				.flatMap(new RestoringCheckingFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
-				.keyBy(0)
-				.flatMap(new KeyedStateCheckingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
-				.keyBy(0)
-				.transform(
-						"custom_operator",
-						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
-						new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
-				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>(EXPECTED_ELEMENTS_ACCUMULATOR));
-
-		restoreAndExecute(
-				env,
-				getResourceFilename("stateful-udf-migration-itcase-flink1.1-savepoint-rocksdb"),
-				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, EXPECTED_SUCCESSFUL_CHECKS));
-	}
-
-	private static class LegacyCheckpointedSource
-			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
-
-		public static String CHECKPOINTED_STRING = "Here be dragons!";
-
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean isRunning = true;
-
-		private final int numElements;
-
-		public LegacyCheckpointedSource(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-
-			synchronized (ctx.getCheckpointLock()) {
-				for (long i = 0; i < numElements; i++) {
-					ctx.collect(new Tuple2<>(i, i));
-				}
-			}
-			while (isRunning) {
-				Thread.sleep(20);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public void restoreState(String state) throws Exception {
-			assertEquals(CHECKPOINTED_STRING, state);
-		}
-
-		@Override
-		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_STRING;
-		}
-	}
-
-	private static class RestoringCheckingSource
-			extends RichSourceFunction<Tuple2<Long, Long>>
-			implements CheckpointedRestoring<String> {
-
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean isRunning = true;
-
-		private final int numElements;
-
-		private String restoredState;
-
-		public RestoringCheckingSource(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-
-			synchronized (ctx.getCheckpointLock()) {
-				for (long i = 0; i < numElements; i++) {
-					ctx.collect(new Tuple2<>(i, i));
-				}
-			}
-
-			while (isRunning) {
-				Thread.sleep(20);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public void restoreState(String state) throws Exception {
-			restoredState = state;
-		}
-	}
-
-	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
-				new Tuple2<>("hello", 42L);
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-		}
-
-		@Override
-		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
-		}
-	}
-
-	public static class RestoringCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements CheckpointedRestoring<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private transient Tuple2<String, Long> restoredState;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
-		}
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
-	}
-
-	public static class LegacyCheckpointedFlatMapWithKeyedState
-			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
-				new Tuple2<>("hello", 42L);
-
-		private final ValueStateDescriptor<Long> stateDescriptor =
-				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-
-			getRuntimeContext().getState(stateDescriptor).update(value.f1);
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-		}
-
-		@Override
-		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return CHECKPOINTED_TUPLE;
-		}
-	}
-
-	public static class RestoringCheckingFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements CheckpointedRestoring<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private transient Tuple2<String, Long> restoredState;
-
-		private final ValueStateDescriptor<Long> stateDescriptor =
-				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
-		}
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-
-			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
-			if (state == null) {
-				throw new RuntimeException("Missing key value state for " + value);
-			}
-
-			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
-	}
-
-	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final ValueStateDescriptor<Long> stateDescriptor =
-				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-
-			getRuntimeContext().getState(stateDescriptor).update(value.f1);
-		}
-	}
-
-	public static class KeyedStateCheckingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final ValueStateDescriptor<Long> stateDescriptor =
-				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter());
-		}
-
-		@Override
-		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
-			out.collect(value);
-
-			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
-			if (state == null) {
-				throw new RuntimeException("Missing key value state for " + value);
-			}
-
-			assertEquals(value.f1, state.value());
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-		}
-	}
-
-	public static class CheckpointedUdfOperator
-			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
-			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		private static final String CHECKPOINTED_STRING = "Oh my, that's nice!";
-
-		public CheckpointedUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
-			super(userFunction);
-		}
-
-		@Override
-		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-			output.emitWatermark(mark);
-		}
-
-		// Flink 1.1
-//		@Override
-//		public StreamTaskState snapshotOperatorState(
-//				long checkpointId, long timestamp) throws Exception {
-//			StreamTaskState result = super.snapshotOperatorState(checkpointId, timestamp);
-//
-//			AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(
-//					checkpointId,
-//					timestamp);
-//
-//			out.writeUTF(CHECKPOINTED_STRING);
-//
-//			result.setOperatorState(out.closeAndGetHandle());
-//
-//			return result;
-//		}
-	}
-
-	public static class RestoringCheckingUdfOperator
-			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
-			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		private String restoredState;
-
-		public RestoringCheckingUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
-			super(userFunction);
-		}
-
-		@Override
-		public void open() throws Exception {
-			super.open();
-		}
-
-		@Override
-		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
-			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
-
-			assertEquals(CheckpointedUdfOperator.CHECKPOINTED_STRING, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-			output.emitWatermark(mark);
-		}
-
-		@Override
-		public void restoreState(FSDataInputStream in) throws Exception {
-			super.restoreState(in);
-
-			DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
-
-			restoredState = streamWrapper.readUTF();
-		}
-	}
-
-	public static class AccumulatorCountingSink<T> extends RichSinkFunction<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final String accumulatorName;
-
-		int count = 0;
-
-		public AccumulatorCountingSink(String accumulatorName) {
-			this.accumulatorName = accumulatorName;
-		}
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-
-			getRuntimeContext().addAccumulator(accumulatorName, new IntCounter());
-		}
-
-		@Override
-		public void invoke(T value) throws Exception {
-			count++;
-			getRuntimeContext().getAccumulator(accumulatorName).add(1);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint
new file mode 100644
index 0000000..a8d19f2
Binary files /dev/null and b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint differ

http://git-wip-us.apache.org/repos/asf/flink/blob/52fb5789/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb
new file mode 100644
index 0000000..548993f
Binary files /dev/null and b/flink-tests/src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb differ


[07/11] flink git commit: [FLINK-5969] Add KafkaConsumerBaseFrom12MigrationTest

Posted by al...@apache.org.
[FLINK-5969] Add KafkaConsumerBaseFrom12MigrationTest

The binary snapshots were created on the Flink 1.2 branch.


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

Branch: refs/heads/release-1.2
Commit: c43fc2a14fa9ece5f3a80c8f25e6d99a2450f671
Parents: 62601b4
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 15:12:14 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:04 2017 +0200

----------------------------------------------------------------------
 ...inkKafkaConsumerBaseFrom11MigrationTest.java | 530 +++++++++++++++++++
 ...inkKafkaConsumerBaseFrom12MigrationTest.java | 487 +++++++++++++++++
 .../FlinkKafkaConsumerBaseMigrationTest.java    | 530 -------------------
 ...ka-consumer-migration-test-flink1.2-snapshot | Bin 0 -> 1022 bytes
 ...migration-test-flink1.2-snapshot-empty-state | Bin 0 -> 240 bytes
 5 files changed, 1017 insertions(+), 530 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c43fc2a1/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
new file mode 100644
index 0000000..1330a6d
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
@@ -0,0 +1,530 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+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 org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.Serializable;
+import java.net.URL;
+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 static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.anyMapOf;
+import static org.mockito.Mockito.doAnswer;
+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 {
+
+	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();
+	}
+
+	@Test
+	public void testRestoreFromFlink11WithEmptyStateNoPartitions() throws Exception {
+		// --------------------------------------------------------------------
+		//   prepare fake states
+		// --------------------------------------------------------------------
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return fetcher;
+					}
+				},
+				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();
+		testHarness.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
+		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();
+							Assert.fail("This should never be called.");
+						}
+
+						@Override
+						public void emitWatermark(Watermark mark) {
+							latch.trigger();
+							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		consumerOperator.close();
+
+		consumerOperator.cancel();
+		runner.interrupt();
+		runner.join();
+
+		Assert.assertNull(error[0]);
+	}
+
+	@Test
+	public void testRestoreFromFlink11WithEmptyStateWithPartitions() throws Exception {
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
+		partitions.add(new KafkaTopicPartition("abc", 13));
+		partitions.add(new KafkaTopicPartition("def", 7));
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return 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.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
+		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();
+							Assert.fail("This should never be called.");
+						}
+
+						@Override
+						public void emitWatermark(Watermark mark) {
+							latch.trigger();
+							Assert.fail("This should never be called.");
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		consumerOperator.close();
+
+		runner.join();
+
+		Assert.assertNull(error[0]);
+	}
+
+	@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() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
+
+				latch.trigger();
+				assertEquals(state1, map);
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
+		partitions.add(new KafkaTopicPartition("abc", 13));
+		partitions.add(new KafkaTopicPartition("def", 7));
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return 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.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot"));
+		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();
+		}
+
+		consumerOperator.close();
+
+		runner.join();
+
+		Assert.assertNull(error[0]);
+	}
+
+	private abstract static 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;
+
+		private final List<KafkaTopicPartition> partitions;
+
+		@SuppressWarnings("unchecked")
+		DummyFlinkKafkaConsumer(
+				FetcherFactory<T> fetcherFactory,
+				List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcherFactory = fetcherFactory;
+			this.partitions = partitions;
+		}
+
+		@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();
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return partitions;
+		}
+	}
+}
+
+/*
+	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/c43fc2a1/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
new file mode 100644
index 0000000..098fcbb
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyMapOf;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.Serializable;
+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 java.util.concurrent.TimeUnit;
+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.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+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-snapshot-empty-state", 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<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return 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) {
+							latch.trigger();
+						}
+					});
+				}
+				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
+	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
+		// --------------------------------------------------------------------
+		//   prepare fake states
+		// --------------------------------------------------------------------
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return fetcher;
+					}
+				},
+				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();
+		testHarness.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
+		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();
+							Assert.fail("This should never be called.");
+						}
+
+						@Override
+						public void emitWatermark(Watermark mark) {
+							latch.trigger();
+							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await(2, TimeUnit.MINUTES);
+		}
+
+		assertTrue("Latch was not triggered within the given timeout.", latch.isTriggered());
+
+		consumerOperator.cancel();
+		consumerOperator.close();
+
+		runner.interrupt();
+		runner.join();
+
+		assertNull("Got error: " + error[0], error[0]);
+	}
+
+	@Test
+	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				latch.trigger();
+				Assert.fail("This should never be called");
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return 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.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
+		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();
+							Assert.fail("This should never be called.");
+						}
+
+						@Override
+						public void emitWatermark(Watermark mark) {
+							latch.trigger();
+							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		consumerOperator.close();
+		runner.interrupt();
+		runner.join();
+
+		assertNull("Got error: " + error[0], error[0]);
+	}
+
+	@Test
+	public void testRestore() throws Exception {
+		// --------------------------------------------------------------------
+		//   prepare fake states
+		// --------------------------------------------------------------------
+
+		final boolean[] verifiedState = new boolean[1];
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer() {
+			@Override
+			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
+
+				latch.trigger();
+				assertEquals(PARTITION_STATE, map);
+				verifiedState[0] = true;
+				return null;
+			}
+		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
+
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+				new FetcherFactory<String>() {
+					private static final long serialVersionUID = -2803131905656983619L;
+
+					@Override
+					public AbstractFetcher<String, ?> createFetcher() {
+						return 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.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
+		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();
+		}
+
+		consumerOperator.close();
+
+		runner.join();
+
+		assertNull("Got error: " + error[0], error[0]);
+
+		assertTrue(verifiedState[0]);
+	}
+
+	private abstract static 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;
+
+		private final List<KafkaTopicPartition> partitions;
+
+		@SuppressWarnings("unchecked")
+		DummyFlinkKafkaConsumer(
+				FetcherFactory<T> fetcherFactory,
+				List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcherFactory = fetcherFactory;
+			this.partitions = partitions;
+		}
+
+		@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();
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return partitions;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c43fc2a1/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
deleted file mode 100644
index fa93138..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
+++ /dev/null
@@ -1,530 +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.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.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-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 org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.Serializable;
-import java.net.URL;
-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 static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyMapOf;
-import static org.mockito.Mockito.doAnswer;
-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 FlinkKafkaConsumerBaseMigrationTest {
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = FlinkKafkaConsumerBaseMigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateNoPartitions() throws Exception {
-		// --------------------------------------------------------------------
-		//   prepare fake states
-		// --------------------------------------------------------------------
-
-		final OneShotLatch latch = new OneShotLatch();
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
-		doAnswer(new Answer() {
-			@Override
-			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
-				latch.trigger();
-				Assert.fail("This should never be called");
-				return null;
-			}
-		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				latch.trigger();
-				Assert.fail("This should never be called");
-				return null;
-			}
-		}).when(fetcher).runFetchLoop();
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
-				new FetcherFactory<String>() {
-					private static final long serialVersionUID = -2803131905656983619L;
-
-					@Override
-					public AbstractFetcher<String, ?> createFetcher() {
-						return fetcher;
-					}
-				},
-				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();
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
-		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();
-							Assert.fail("This should never be called.");
-						}
-
-						@Override
-						public void emitWatermark(Watermark mark) {
-							latch.trigger();
-							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		consumerOperator.close();
-
-		consumerOperator.cancel();
-		runner.interrupt();
-		runner.join();
-
-		Assert.assertNull(error[0]);
-	}
-
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateWithPartitions() throws Exception {
-		final OneShotLatch latch = new OneShotLatch();
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
-		doAnswer(new Answer() {
-			@Override
-			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
-				latch.trigger();
-				Assert.fail("This should never be called");
-				return null;
-			}
-		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				latch.trigger();
-				return null;
-			}
-		}).when(fetcher).runFetchLoop();
-
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
-				new FetcherFactory<String>() {
-					private static final long serialVersionUID = -2803131905656983619L;
-
-					@Override
-					public AbstractFetcher<String, ?> createFetcher() {
-						return 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.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
-		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();
-							Assert.fail("This should never be called.");
-						}
-
-						@Override
-						public void emitWatermark(Watermark mark) {
-							latch.trigger();
-							Assert.fail("This should never be called.");
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		consumerOperator.close();
-
-		runner.join();
-
-		Assert.assertNull(error[0]);
-	}
-
-	@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() {
-			@Override
-			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
-				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
-
-				latch.trigger();
-				assertEquals(state1, map);
-				return null;
-			}
-		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
-
-
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
-				new FetcherFactory<String>() {
-					private static final long serialVersionUID = -2803131905656983619L;
-
-					@Override
-					public AbstractFetcher<String, ?> createFetcher() {
-						return 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.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot"));
-		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();
-		}
-
-		consumerOperator.close();
-
-		runner.join();
-
-		Assert.assertNull(error[0]);
-	}
-
-	private abstract static 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;
-
-		private final List<KafkaTopicPartition> partitions;
-
-		@SuppressWarnings("unchecked")
-		DummyFlinkKafkaConsumer(
-				FetcherFactory<T> fetcherFactory,
-				List<KafkaTopicPartition> partitions) {
-			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-			this.fetcherFactory = fetcherFactory;
-			this.partitions = partitions;
-		}
-
-		@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();
-		}
-
-		@Override
-		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-			return partitions;
-		}
-	}
-}
-
-/*
-	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/c43fc2a1/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot
new file mode 100644
index 0000000..f0be11a
Binary files /dev/null and b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot differ

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


[06/11] flink git commit: [FLINK-5969] Also snapshot legacy state in operator test harness

Posted by al...@apache.org.
[FLINK-5969] Also snapshot legacy state in operator test harness


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

Branch: refs/heads/release-1.2
Commit: c89d4b43213d7d3e2ad184787061fe3178be5691
Parents: c43fc2a
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 17:13:49 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 13:50:04 2017 +0200

----------------------------------------------------------------------
 .../util/AbstractStreamOperatorTestHarness.java   | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c89d4b43/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
index 2df4efd..baced53 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java
@@ -471,9 +471,25 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 		OperatorStateHandle opManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateManagedFuture());
 		OperatorStateHandle opRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateRawFuture());
 
+		// also snapshot legacy state, if any
+		StreamStateHandle legacyStateHandle = null;
+
+		if (operator instanceof StreamCheckpointedOperator) {
+
+			CheckpointStreamFactory streamFactory = stateBackend.createStreamFactory(
+					new JobID(),
+					"test_op");
+
+			final CheckpointStreamFactory.CheckpointStateOutputStream outStream =
+					streamFactory.createCheckpointStateOutputStream(checkpointId, timestamp);
+
+				((StreamCheckpointedOperator) operator).snapshotState(outStream, checkpointId, timestamp);
+				legacyStateHandle = outStream.closeAndGetHandle();
+		}
+
 		return new OperatorStateHandles(
 			0,
-			null,
+			legacyStateHandle,
 			keyedManaged != null ? Collections.singletonList(keyedManaged) : null,
 			keyedRaw != null ? Collections.singletonList(keyedRaw) : null,
 			opManaged != null ? Collections.singletonList(opManaged) : null,