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 14:28:07 UTC

[01/13] flink git commit: [FLINK-5969] Fix restore from empty state in KafkaConsumerBase

Repository: flink
Updated Branches:
  refs/heads/master 72dfce40b -> 821ec80d7


[FLINK-5969] Fix restore from empty state in KafkaConsumerBase


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

Branch: refs/heads/master
Commit: 0ecb5d0050b84ba48105836288d43ce4c4749459
Parents: 44e472f
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Apr 27 10:49:05 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 16:24:26 2017 +0200

----------------------------------------------------------------------
 .../flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0ecb5d00/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
index a35e710..4a05efa 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -516,7 +516,8 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Using the following offsets: {}", restoredState);
 				}
-			} else if (restoredState.isEmpty()) {
+			}
+			if (restoredState != null && restoredState.isEmpty()) {
 				restoredState = null;
 			}
 		} else {


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

Posted by al...@apache.org.
[FLINK-6353] Fix legacy user-state restore from 1.2

State that was checkpointed using Checkpointed (on a user function)
could 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/cced07a2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cced07a2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cced07a2

Branch: refs/heads/master
Commit: cced07a28622016ca1ee2d5b316423701c9a986c
Parents: 72dfce4
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 16:24:26 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/cced07a2/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 2fa1e38..14857de 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
@@ -154,12 +154,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());
@@ -171,7 +173,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) {


[13/13] 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/821ec80d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/821ec80d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/821ec80d

Branch: refs/heads/master
Commit: 821ec80d72308746aed307498be157b58b7b65e9
Parents: fb7793f
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 16:25:58 2017 +0200

----------------------------------------------------------------------
 .../AbstractKeyedCEPPatternOperator.java        |  11 +-
 .../cep/operator/CEPFrom12MigrationTest.java    | 492 +++++++++++++++++++
 .../cep/operator/CEPMigration12to13Test.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
 6 files changed, 500 insertions(+), 483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/821ec80d/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index 5f24875..cd62c0d 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -42,6 +42,7 @@ import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Migration;
 import org.apache.flink.util.OutputTag;
 import org.apache.flink.util.Preconditions;
 
@@ -323,9 +324,13 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 
 	@Override
 	public void restoreState(FSDataInputStream in) throws Exception {
-		// this is the flag indicating if we have udf
-		// state to restore (not needed here)
-		in.read();
+		if (in instanceof Migration) {
+			// absorb the introduced byte from the migration stream
+			int hasUdfState = in.read();
+			if (hasUdfState == 1) {
+				throw new Exception("Found UDF state but CEPOperator is not an UDF operator.");
+			}
+		}
 
 		DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(in);
 		timerService = getInternalTimerService(

http://git-wip-us.apache.org/repos/asf/flink/blob/821ec80d/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..2f7cdeb
--- /dev/null
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPFrom12MigrationTest.java
@@ -0,0 +1,492 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.cep.operator;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.cep.Event;
+import org.apache.flink.cep.SubEvent;
+import org.apache.flink.cep.nfa.NFA;
+import org.apache.flink.cep.nfa.compiler.NFACompiler;
+import org.apache.flink.cep.pattern.Pattern;
+import org.apache.flink.cep.pattern.conditions.SimpleCondition;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.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(),
+								null,
+								true),
+						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(),
+								null,
+								true),
+						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(),
+								null,
+								true),
+						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(),
+								null,
+								true),
+						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(),
+								null,
+								true),
+						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(),
+								null,
+								true),
+						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 extends SimpleCondition<Event> {
+		private static final long serialVersionUID = 5726188262756267490L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("start");
+		}
+	}
+
+	private static class MiddleFilter extends SimpleCondition<SubEvent> {
+		private static final long serialVersionUID = 6215754202506583964L;
+
+		@Override
+		public boolean filter(SubEvent value) throws Exception {
+			return value.getVolume() > 5.0;
+		}
+	}
+
+	private static class EndFilter extends SimpleCondition<Event> {
+		private static final long serialVersionUID = 7056763917392056548L;
+
+		@Override
+		public boolean filter(Event value) throws Exception {
+			return value.getName().equals("end");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/821ec80d/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
deleted file mode 100644
index 8249535..0000000
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration12to13Test.java
+++ /dev/null
@@ -1,480 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.cep.operator;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.cep.Event;
-import org.apache.flink.cep.SubEvent;
-import org.apache.flink.cep.nfa.NFA;
-import org.apache.flink.cep.nfa.compiler.NFACompiler;
-import org.apache.flink.cep.pattern.Pattern;
-import org.apache.flink.cep.pattern.conditions.SimpleCondition;
-import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.OperatorStateHandle;
-import org.apache.flink.runtime.state.StreamStateHandle;
-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.junit.Test;
-
-import java.io.FileInputStream;
-import java.io.ObjectInputStream;
-import java.net.URL;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class CEPMigration12to13Test {
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = CEPMigration12to13Test.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	@Test
-	public void testMigrationAfterBranchingPattern() 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);
-
-		// uncomment these lines for regenerating the snapshot on Flink 1.2
-//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-//			new KeyedOneInputStreamOperatorTestHarness<>(
-//				new KeyedCEPPatternOperator<>(
-//					Event.createTypeSerializer(),
-//					false,
-//					keySelector,
-//					IntSerializer.INSTANCE,
-//					new NFAFactory(),
-//					true),
-//				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));
-//		// simulate snapshot/restore with empty element queue but NFA state
-//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
-//		FileOutputStream out = new FileOutputStream(
-//				"src/test/resources/cep-branching-snapshot-1.2");
-//		ObjectOutputStream oos = new ObjectOutputStream(out);
-//		oos.writeObject(snapshot.getOperatorChainIndex());
-//		oos.writeObject(snapshot.getLegacyOperatorState());
-//		oos.writeObject(snapshot.getManagedKeyedState());
-//		oos.writeObject(snapshot.getRawKeyedState());
-//		oos.writeObject(snapshot.getManagedOperatorState());
-//		oos.writeObject(snapshot.getRawOperatorState());
-//		out.close();
-//		harness.close();
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-			new KeyedOneInputStreamOperatorTestHarness<>(
-				new KeyedCEPPatternOperator<>(
-					Event.createTypeSerializer(),
-					false,
-					keySelector,
-					IntSerializer.INSTANCE,
-					new NFAFactory(),
-					null,
-					true),
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-
-		harness.setup();
-		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
-			"cep-branching-snapshot-1.2")));
-		final OperatorStateHandles snapshot = new OperatorStateHandles(
-			(int) ois.readObject(),
-			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject()
-		);
-		harness.initializeState(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();
-	}
-
-	@Test
-	public void testStartingNewPatternAfterMigration() 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);
-
-		// uncomment these lines for regenerating the snapshot on Flink 1.2
-//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-//			new KeyedOneInputStreamOperatorTestHarness<>(
-//				new KeyedCEPPatternOperator<>(
-//					Event.createTypeSerializer(),
-//					false,
-//					keySelector,
-//					IntSerializer.INSTANCE,
-//					new NFAFactory(),
-//					true),
-//				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));
-//		// simulate snapshot/restore with empty element queue but NFA state
-//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
-//		FileOutputStream out = new FileOutputStream(
-//				"src/test/resources/cep-starting-snapshot-1.2");
-//		ObjectOutputStream oos = new ObjectOutputStream(out);
-//		oos.writeObject(snapshot.getOperatorChainIndex());
-//		oos.writeObject(snapshot.getLegacyOperatorState());
-//		oos.writeObject(snapshot.getManagedKeyedState());
-//		oos.writeObject(snapshot.getRawKeyedState());
-//		oos.writeObject(snapshot.getManagedOperatorState());
-//		oos.writeObject(snapshot.getRawOperatorState());
-//		out.close();
-//		harness.close();
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-			new KeyedOneInputStreamOperatorTestHarness<>(
-				new KeyedCEPPatternOperator<>(
-					Event.createTypeSerializer(),
-					false,
-					keySelector,
-					IntSerializer.INSTANCE,
-					new NFAFactory(),
-					null,
-					true),
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-
-		harness.setup();
-		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
-			"cep-starting-snapshot-1.2")));
-		final OperatorStateHandles snapshot = new OperatorStateHandles(
-			(int) ois.readObject(),
-			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject()
-		);
-		harness.initializeState(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();
-	}
-
-	@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);
-
-		// uncomment these lines for regenerating the snapshot on Flink 1.2
-//		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-//			new KeyedOneInputStreamOperatorTestHarness<>(
-//				new KeyedCEPPatternOperator<>(
-//					Event.createTypeSerializer(),
-//					false,
-//					keySelector,
-//					IntSerializer.INSTANCE,
-//					new SinglePatternNFAFactory(),
-//					true),
-//				keySelector,
-//				BasicTypeInfo.INT_TYPE_INFO);
-//
-//		harness.setup();
-//		harness.open();
-//		harness.processWatermark(new Watermark(5));
-//		// simulate snapshot/restore with empty element queue but NFA state
-//		OperatorStateHandles snapshot = harness.snapshot(1, 1);
-//		FileOutputStream out = new FileOutputStream(
-//				"src/test/resources/cep-single-pattern-snapshot-1.2");
-//		ObjectOutputStream oos = new ObjectOutputStream(out);
-//		oos.writeObject(snapshot.getOperatorChainIndex());
-//		oos.writeObject(snapshot.getLegacyOperatorState());
-//		oos.writeObject(snapshot.getManagedKeyedState());
-//		oos.writeObject(snapshot.getRawKeyedState());
-//		oos.writeObject(snapshot.getManagedOperatorState());
-//		oos.writeObject(snapshot.getRawOperatorState());
-//		out.close();
-//		harness.close();
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness =
-			new KeyedOneInputStreamOperatorTestHarness<>(
-				new KeyedCEPPatternOperator<>(
-					Event.createTypeSerializer(),
-					false,
-					keySelector,
-					IntSerializer.INSTANCE,
-					new SinglePatternNFAFactory(),
-					null,
-					true),
-				keySelector,
-				BasicTypeInfo.INT_TYPE_INFO);
-
-		harness.setup();
-		final ObjectInputStream ois = new ObjectInputStream(new FileInputStream(getResourceFilename(
-			"cep-single-pattern-snapshot-1.2")));
-		final OperatorStateHandles snapshot = new OperatorStateHandles(
-			(int) ois.readObject(),
-			(StreamStateHandle) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<KeyedStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject(),
-			(Collection<OperatorStateHandle>) ois.readObject()
-		);
-		harness.initializeState(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 extends SimpleCondition<Event> {
-		private static final long serialVersionUID = 5726188262756267490L;
-
-		@Override
-		public boolean filter(Event value) throws Exception {
-			return value.getName().equals("start");
-		}
-	}
-
-	private static class MiddleFilter extends SimpleCondition<SubEvent> {
-		private static final long serialVersionUID = 6215754202506583964L;
-
-		@Override
-		public boolean filter(SubEvent value) throws Exception {
-			return value.getVolume() > 5.0;
-		}
-	}
-
-	private static class EndFilter extends SimpleCondition<Event> {
-		private static final long serialVersionUID = 7056763917392056548L;
-
-		@Override
-		public boolean filter(Event value) throws Exception {
-			return value.getName().equals("end");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/821ec80d/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/821ec80d/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/821ec80d/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


[12/13] 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/84eea722
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/84eea722
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/84eea722

Branch: refs/heads/master
Commit: 84eea72295eda5e7289deb5221c7b990b7b65883
Parents: e40f2e1
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 16:25:57 2017 +0200

----------------------------------------------------------------------
 .../util/AbstractStreamOperatorTestHarness.java      | 15 +++++++++++++--
 1 file changed, 13 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/84eea722/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 4a9463a..7a8488f 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
@@ -509,9 +509,21 @@ 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) {
+
+			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,
@@ -523,7 +535,6 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 	 * the operator implements this interface.
 	 */
 	@Deprecated
-	@SuppressWarnings("deprecation")
 	public StreamStateHandle snapshotLegacy(long checkpointId, long timestamp) throws Exception {
 
 		CheckpointStreamFactory.CheckpointStateOutputStream outStream = stateBackend.createStreamFactory(


[04/13] flink git commit: [FLINK-5969] Remove watermark callback service

Posted by al...@apache.org.
[FLINK-5969] Remove watermark callback service

In Flink 1.2 we only wrote timers to key-grouped state streams. With the
addition of the watermark callback service we started to also write the
watermark callbacks to the key-grouped streams. This breaks backwards
compatibility with saveoints taken on Flink 1.2

This replaces usage of the watermark callback service by setting a
regular timer for "current watermark + 1". These timers will fire as
soon as the watermark advances, thus simulating the watermark callback
service.


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

Branch: refs/heads/master
Commit: 6f8b3c6158a87c14f2fdb3446092df367131e194
Parents: cced07a
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Apr 25 14:56:14 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 16:24:26 2017 +0200

----------------------------------------------------------------------
 .../AbstractKeyedCEPPatternOperator.java        | 134 ++++---
 .../flink/cep/operator/CEPOperatorTest.java     |  10 +-
 .../api/operators/AbstractStreamOperator.java   |  20 --
 .../operators/InternalTimeServiceManager.java   |  35 +-
 .../InternalWatermarkCallbackService.java       | 296 ----------------
 .../operators/AbstractStreamOperatorTest.java   | 346 -------------------
 .../util/AbstractStreamOperatorTestHarness.java |   9 -
 7 files changed, 85 insertions(+), 765 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
index b232dbb..5f24875 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java
@@ -30,13 +30,15 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.migration.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.api.operators.InternalWatermarkCallbackService;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.streaming.api.operators.CheckpointedRestoringOperator;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OnWatermarkCallback;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -62,7 +64,7 @@ import java.util.PriorityQueue;
  */
 public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 	extends AbstractStreamOperator<OUT>
-	implements OneInputStreamOperator<IN, OUT>, CheckpointedRestoringOperator {
+	implements OneInputStreamOperator<IN, OUT>, Triggerable<KEY, VoidNamespace>, CheckpointedRestoringOperator {
 
 	private static final long serialVersionUID = -4166778210774160757L;
 
@@ -89,6 +91,8 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 	private final PriorityQueueFactory<StreamRecord<IN>> priorityQueueFactory = new PriorityQueueStreamRecordFactory<>();
 	private final NFACompiler.NFAFactory<IN> nfaFactory;
 
+	private transient InternalTimerService<VoidNamespace> timerService;
+
 	/**
 	 * {@link OutputTag} to use for late arriving events. Elements for which
 	 * {@code window.maxTimestamp + allowedLateness} is smaller than the current watermark will
@@ -159,53 +163,14 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 	public void open() throws Exception {
 		super.open();
 
-		final InternalWatermarkCallbackService<KEY> watermarkCallbackService = getInternalWatermarkCallbackService();
-
-		watermarkCallbackService.setWatermarkCallback(
-			new OnWatermarkCallback<KEY>() {
-
-				@Override
-				public void onWatermark(KEY key, Watermark watermark) throws IOException {
-
-					// 1) get the queue of pending elements for the key and the corresponding NFA,
-					// 2) process the pending elements in event time order by feeding them in the NFA
-					// 3) advance the time to the current watermark, so that expired patterns are discarded.
-					// 4) update the stored state for the key, by only storing the new NFA and priority queue iff they
-					//		have state to be used later.
-					// 5) update the last seen watermark.
-
-					// STEP 1
-					PriorityQueue<StreamRecord<IN>> priorityQueue = getPriorityQueue();
-					NFA<IN> nfa = getNFA();
-
-					// STEP 2
-					while (!priorityQueue.isEmpty() && priorityQueue.peek().getTimestamp() <= watermark.getTimestamp()) {
-						StreamRecord<IN> streamRecord = priorityQueue.poll();
-						processEvent(nfa, streamRecord.getValue(), streamRecord.getTimestamp());
-					}
-
-					// STEP 3
-					advanceTime(nfa, watermark.getTimestamp());
-
-					// STEP 4
-					updatePriorityQueue(priorityQueue);
-					updateNFA(nfa);
-
-					if (priorityQueue.isEmpty() && nfa.isEmpty()) {
-						watermarkCallbackService.unregisterKeyFromWatermarkCallback(key);
-					}
-
-					// STEP 5
-					updateLastSeenWatermark(watermark);
-				}
-			},
-			keySerializer
-		);
+		timerService = getInternalTimerService(
+				"watermark-callbacks",
+				VoidNamespaceSerializer.INSTANCE,
+				this);
 	}
 
 	@Override
 	public void processElement(StreamRecord<IN> element) throws Exception {
-
 		if (isProcessingTime) {
 			// there can be no out of order elements in processing time
 			NFA<IN> nfa = getNFA();
@@ -223,7 +188,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 				// we have an event with a valid timestamp, so
 				// we buffer it until we receive the proper watermark.
 
-				getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(keySelector.getKey(element.getValue()));
+				saveRegisterWatermarkTimer();
 
 				PriorityQueue<StreamRecord<IN>> priorityQueue = getPriorityQueue();
 				if (getExecutionConfig().isObjectReuseEnabled()) {
@@ -239,8 +204,62 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 		}
 	}
 
-	private void updateLastSeenWatermark(Watermark watermark) {
-		this.lastWatermark = watermark.getTimestamp();
+	/**
+	 * Registers a timer for {@code current watermark + 1}, this means that we get triggered
+	 * whenever the watermark advances, which is what we want for working off the queue of
+	 * buffered elements.
+	 */
+	private void saveRegisterWatermarkTimer() {
+		long currentWatermark = timerService.currentWatermark();
+		// protect against overflow
+		if (currentWatermark + 1 > currentWatermark) {
+			timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, currentWatermark + 1);
+		}
+	}
+
+	long count = 0;
+	@Override
+	public void onEventTime(InternalTimer<KEY, VoidNamespace> timer) throws Exception {
+
+		// 1) get the queue of pending elements for the key and the corresponding NFA,
+		// 2) process the pending elements in event time order by feeding them in the NFA
+		// 3) advance the time to the current watermark, so that expired patterns are discarded.
+		// 4) update the stored state for the key, by only storing the new NFA and priority queue iff they
+		//		have state to be used later.
+		// 5) update the last seen watermark.
+
+		// STEP 1
+		PriorityQueue<StreamRecord<IN>> priorityQueue = getPriorityQueue();
+		NFA<IN> nfa = getNFA();
+
+		// STEP 2
+		while (!priorityQueue.isEmpty() && priorityQueue.peek().getTimestamp() <= timerService.currentWatermark()) {
+			StreamRecord<IN> streamRecord = priorityQueue.poll();
+			processEvent(nfa, streamRecord.getValue(), streamRecord.getTimestamp());
+		}
+
+		// STEP 3
+		advanceTime(nfa, timerService.currentWatermark());
+
+		// STEP 4
+		updatePriorityQueue(priorityQueue);
+		updateNFA(nfa);
+
+		if (!priorityQueue.isEmpty() || !nfa.isEmpty()) {
+			saveRegisterWatermarkTimer();
+		}
+
+		// STEP 5
+		updateLastSeenWatermark(timerService.currentWatermark());
+	}
+
+	@Override
+	public void onProcessingTime(InternalTimer<KEY, VoidNamespace> timer) throws Exception {
+		// not used
+	}
+
+	private void updateLastSeenWatermark(long timestamp) {
+		this.lastWatermark = timestamp;
 	}
 
 	/**
@@ -309,12 +328,17 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 		in.read();
 
 		DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(in);
-		InternalWatermarkCallbackService<KEY> watermarkCallbackService = getInternalWatermarkCallbackService();
+		timerService = getInternalTimerService(
+				"watermark-callbacks",
+				VoidNamespaceSerializer.INSTANCE,
+				this);
 
 		if (migratingFromOldKeyedOperator) {
 			int numberEntries = inputView.readInt();
 			for (int i = 0; i <numberEntries; i++) {
-				watermarkCallbackService.registerKeyForWatermarkCallback(keySerializer.deserialize(inputView));
+				KEY key = keySerializer.deserialize(inputView);
+				setCurrentKey(key);
+				saveRegisterWatermarkTimer();
 			}
 		} else {
 
@@ -342,10 +366,8 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT>
 				// this is relevant only for event/ingestion time
 
 				// need to work around type restrictions
-				InternalWatermarkCallbackService rawWatermarkCallbackService =
-					(InternalWatermarkCallbackService) watermarkCallbackService;
-
-				rawWatermarkCallbackService.registerKeyForWatermarkCallback((byte) 0);
+				setCurrentKey((byte) 0);
+				saveRegisterWatermarkTimer();
 			}
 			ois.close();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
index 8465bc3..4048bc2 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
@@ -307,7 +307,7 @@ public class CEPOperatorTest extends TestLogger {
 		// there must be 2 keys 42, 43 registered for the watermark callback
 		// all the seen elements must be in the priority queues but no NFA yet.
 
-		assertEquals(2L, harness.numKeysForWatermarkCallback());
+		assertEquals(2L, harness.numEventTimeTimers());
 		assertEquals(4L, operator.getPQSize(42));
 		assertEquals(1L, operator.getPQSize(43));
 		assertTrue(!operator.hasNonEmptyNFA(42));
@@ -322,7 +322,7 @@ public class CEPOperatorTest extends TestLogger {
 		// one element in PQ for 42 (the barfoo) as it arrived early
 		// for 43 the element entered the NFA and the PQ is empty
 
-		assertEquals(2L, harness.numKeysForWatermarkCallback());
+		assertEquals(2L, harness.numEventTimeTimers());
 		assertTrue(operator.hasNonEmptyNFA(42));
 		assertEquals(1L, operator.getPQSize(42));
 		assertTrue(operator.hasNonEmptyNFA(43));
@@ -336,7 +336,7 @@ public class CEPOperatorTest extends TestLogger {
 
 		// now we have 1 key because the 43 expired and was removed.
 		// 42 is still there due to startEvent2
-		assertEquals(1L, harness.numKeysForWatermarkCallback());
+		assertEquals(1L, harness.numEventTimeTimers());
 		assertTrue(operator.hasNonEmptyNFA(42));
 		assertTrue(!operator.hasNonEmptyPQ(42));
 		assertTrue(!operator.hasNonEmptyNFA(43));
@@ -355,7 +355,7 @@ public class CEPOperatorTest extends TestLogger {
 
 		assertTrue(!operator.hasNonEmptyNFA(42));
 		assertTrue(!operator.hasNonEmptyPQ(42));
-		assertEquals(0L, harness.numKeysForWatermarkCallback());
+		assertEquals(0L, harness.numEventTimeTimers());
 
 		verifyPattern(harness.getOutput().poll(), startEvent2, middleEvent2, endEvent2);
 		verifyPattern(harness.getOutput().poll(), startEvent2, middleEvent3, endEvent2);
@@ -430,7 +430,7 @@ public class CEPOperatorTest extends TestLogger {
 		// the pattern expired
 		assertTrue(!operator.hasNonEmptyNFA(42));
 
-		assertEquals(0L, harness.numKeysForWatermarkCallback());
+		assertEquals(0L, harness.numEventTimeTimers());
 		assertTrue(!operator.hasNonEmptyPQ(42));
 		assertTrue(!operator.hasNonEmptyPQ(43));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index 1ef8614..1850007 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -907,20 +907,6 @@ public abstract class AbstractStreamOperator<OUT>
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Returns an {@link InternalWatermarkCallbackService} which  allows to register a
-	 * {@link OnWatermarkCallback} and multiple keys, for which
-	 * the callback will be invoked every time a new {@link Watermark} is received.
-	 *
-	 * <p><b>NOTE: </b> This service is only available to <b>keyed</b> operators.
-	 */
-	public <K> InternalWatermarkCallbackService<K> getInternalWatermarkCallbackService() {
-		checkTimerServiceInitialization();
-
-		InternalTimeServiceManager<K, ?> keyedTimeServiceHandler = (InternalTimeServiceManager<K, ?>) timeServiceManager;
-		return keyedTimeServiceHandler.getWatermarkCallbackService();
-	}
-
-	/**
 	 * Returns a {@link InternalTimerService} that can be used to query current processing time
 	 * and event time and to set timers. An operator can have several timer services, where
 	 * each has its own namespace serializer. Timer services are differentiated by the string
@@ -998,10 +984,4 @@ public abstract class AbstractStreamOperator<OUT>
 		return timeServiceManager == null ? 0 :
 			timeServiceManager.numEventTimeTimers();
 	}
-
-	@VisibleForTesting
-	public int numKeysForWatermarkCallback() {
-		return timeServiceManager == null ? 0 :
-			timeServiceManager.numKeysForWatermarkCallback();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java
index 5886468..ff5164d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java
@@ -34,8 +34,8 @@ import org.apache.flink.util.Preconditions;
 
 /**
  * An entity keeping all the time-related services available to all operators extending the
- * {@link AbstractStreamOperator}. These are the different {@link HeapInternalTimerService timer services}
- * and the {@link InternalWatermarkCallbackService}.
+ * {@link AbstractStreamOperator}. Right now, this is only a
+ * {@link HeapInternalTimerService timer services}.
  *
  * <b>NOTE:</b> These services are only available to keyed operators.
  *
@@ -52,7 +52,6 @@ class InternalTimeServiceManager<K, N> {
 	private final ProcessingTimeService processingTimeService;
 
 	private final Map<String, HeapInternalTimerService<K, N>> timerServices;
-	private final InternalWatermarkCallbackService<K> watermarkCallbackService;
 
 	InternalTimeServiceManager(
 			int totalKeyGroups,
@@ -68,16 +67,6 @@ class InternalTimeServiceManager<K, N> {
 		this.processingTimeService = Preconditions.checkNotNull(processingTimeService);
 
 		this.timerServices = new HashMap<>();
-		this.watermarkCallbackService = new InternalWatermarkCallbackService<>(totalKeyGroups, localKeyGroupRange, keyContext);
-	}
-
-	/**
-	 * Returns an {@link InternalWatermarkCallbackService} which  allows to register a
-	 * {@link OnWatermarkCallback} and multiple keys, for which
-	 * the callback will be invoked every time a new {@link Watermark} is received.
-	 */
-	public InternalWatermarkCallbackService<K> getWatermarkCallbackService() {
-		return watermarkCallbackService;
 	}
 
 	/**
@@ -117,7 +106,6 @@ class InternalTimeServiceManager<K, N> {
 		for (HeapInternalTimerService<?, ?> service : timerServices.values()) {
 			service.advanceWatermark(watermark.getTimestamp());
 		}
-		watermarkCallbackService.invokeOnWatermarkCallback(watermark);
 	}
 
 	//////////////////				Fault Tolerance Methods				///////////////////
@@ -132,15 +120,6 @@ class InternalTimeServiceManager<K, N> {
 			stream.writeUTF(serviceName);
 			timerService.snapshotTimersForKeyGroup(stream, keyGroupIdx);
 		}
-
-		// write a byte indicating if there was a key
-		// registry service instantiated (1) or not (0).
-		if (watermarkCallbackService != null) {
-			stream.writeByte(1);
-			watermarkCallbackService.snapshotKeysForKeyGroup(stream, keyGroupIdx);
-		} else {
-			stream.writeByte(0);
-		}
 	}
 
 	public void restoreStateForKeyGroup(DataInputViewStreamWrapper stream, int keyGroupIdx,
@@ -161,11 +140,6 @@ class InternalTimeServiceManager<K, N> {
 			}
 			timerService.restoreTimersForKeyGroup(stream, keyGroupIdx, userCodeClassLoader);
 		}
-
-		byte hadKeyRegistry = stream.readByte();
-		if (hadKeyRegistry == 1) {
-			watermarkCallbackService.restoreKeysForKeyGroup(stream, keyGroupIdx, userCodeClassLoader);
-		}
 	}
 
 	////////////////////			Methods used ONLY IN TESTS				////////////////////
@@ -187,9 +161,4 @@ class InternalTimeServiceManager<K, N> {
 		}
 		return count;
 	}
-
-	@VisibleForTesting
-	public int numKeysForWatermarkCallback() {
-		return watermarkCallbackService.numKeysForWatermarkCallback();
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalWatermarkCallbackService.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalWatermarkCallbackService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalWatermarkCallbackService.java
deleted file mode 100644
index 441835e..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalWatermarkCallbackService.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
-import org.apache.flink.runtime.state.KeyGroupsList;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.util.InstantiationUtil;
-
-/**
- * The watermark callback service allows to register a {@link OnWatermarkCallback OnWatermarkCallback}
- * and multiple keys, for which the callback will be invoked every time a new {@link Watermark} is received
- * (after the registration of the key).
- *
- * <p><b>NOTE: </b> This service is only available to <b>keyed</b> operators.
- *
- *  @param <K> The type of key returned by the {@code KeySelector}.
- */
-@Internal
-public class InternalWatermarkCallbackService<K> {
-
-	////////////			Information about the keyed state				//////////
-
-	private final KeyGroupsList localKeyGroupRange;
-	private final int totalKeyGroups;
-	private final int localKeyGroupRangeStartIdx;
-
-	private final KeyContext keyContext;
-
-	/**
-	 * An array of sets of keys keeping the registered keys split
-	 * by the key-group they belong to. Each key-group has one set.
-	 */
-	private final Set<K>[] registeredKeysByKeyGroup;
-
-	/**
-	 * An array of sets of keys keeping the keys "to delete" split
-	 * by the key-group they belong to. Each key-group has one set.
-	 *
-	 * <p>This is used to avoid potential concurrent modification
-	 * exception when deleting keys from inside the
-	 * {@link #invokeOnWatermarkCallback(Watermark)}.
-	 */
-	private final Set<K>[] deletedKeysByKeyGroup;
-
-	/** A serializer for the registered keys. */
-	private TypeSerializer<K> keySerializer;
-
-	/**
-	 * The {@link OnWatermarkCallback} to be invoked for each
-	 * registered key upon reception of the watermark.
-	 */
-	private OnWatermarkCallback<K> callback;
-
-	public InternalWatermarkCallbackService(int totalKeyGroups, KeyGroupsList localKeyGroupRange, KeyContext keyContext) {
-
-		this.totalKeyGroups = totalKeyGroups;
-		this.localKeyGroupRange = checkNotNull(localKeyGroupRange);
-		this.keyContext = checkNotNull(keyContext);
-
-		// find the starting index of the local key-group range
-		int startIdx = Integer.MAX_VALUE;
-		for (Integer keyGroupIdx : localKeyGroupRange) {
-			startIdx = Math.min(keyGroupIdx, startIdx);
-		}
-		this.localKeyGroupRangeStartIdx = startIdx;
-
-		// the list of ids of the key-groups this task is responsible for
-		int localKeyGroups = this.localKeyGroupRange.getNumberOfKeyGroups();
-		this.registeredKeysByKeyGroup = new Set[localKeyGroups];
-		this.deletedKeysByKeyGroup = new Set[localKeyGroups];
-	}
-
-	/**
-	 * Registers a {@link OnWatermarkCallback} with the current {@link InternalWatermarkCallbackService} service.
-	 * Before this method is called and the callback is set, the service is unusable.
-	 *
-	 * @param watermarkCallback The callback to be registered.
-	 * @param keySerializer A serializer for the registered keys.
-	 */
-	public void setWatermarkCallback(OnWatermarkCallback<K> watermarkCallback, TypeSerializer<K> keySerializer) {
-		if (callback == null) {
-			this.keySerializer = keySerializer;
-			this.callback = watermarkCallback;
-		} else {
-			throw new RuntimeException("The watermark callback has already been initialized.");
-		}
-	}
-
-	/**
-	 * Registers a key with the service. This will lead to the {@link OnWatermarkCallback}
-	 * being invoked for this key upon reception of each subsequent watermark.
-	 *
-	 * @param key The key to be registered.
-	 */
-	public boolean registerKeyForWatermarkCallback(K key) {
-		return getRegisteredKeysForKeyGroup(key).add(key);
-	}
-
-	/**
-	 * Unregisters the provided key from the service.
-	 *
-	 * @param key The key to be unregistered.
-	 */
-	public boolean unregisterKeyFromWatermarkCallback(K key) {
-		return getDeletedKeysForKeyGroup(key).add(key);
-	}
-
-	/**
-	 * Invokes the registered callback for all the registered keys.
-	 *
-	 * @param watermark The watermark that triggered the invocation.
-	 */
-	public void invokeOnWatermarkCallback(Watermark watermark) throws IOException {
-		// clean up any keys registered for deletion before calling the callback
-		cleanupRegisteredKeys();
-
-		if (callback != null) {
-			for (Set<K> keySet : registeredKeysByKeyGroup) {
-				if (keySet != null) {
-					for (K key : keySet) {
-						keyContext.setCurrentKey(key);
-						callback.onWatermark(key, watermark);
-					}
-				}
-			}
-		}
-	}
-
-	/**
-	 * Does the actual deletion of any keys registered for deletion using the
-	 * {@link #unregisterKeyFromWatermarkCallback(Object)}.
-	 */
-	private void cleanupRegisteredKeys() {
-		for (int keyGroupIdx = 0; keyGroupIdx < registeredKeysByKeyGroup.length; keyGroupIdx++) {
-
-			Set<K> deletedKeys = deletedKeysByKeyGroup[keyGroupIdx];
-			if (deletedKeys != null) {
-
-				Set<K> registeredKeys = registeredKeysByKeyGroup[keyGroupIdx];
-				if (registeredKeys != null) {
-
-					registeredKeys.removeAll(deletedKeys);
-					if (registeredKeys.isEmpty()) {
-						registeredKeysByKeyGroup[keyGroupIdx] = null;
-					}
-				}
-				deletedKeysByKeyGroup[keyGroupIdx] = null;
-			}
-		}
-	}
-
-	/**
-	 * Retrieve the set of keys for the key-group this key belongs to.
-	 *
-	 * @param key the key whose key-group we are searching.
-	 * @return the set of registered keys for the key-group.
-	 */
-	private Set<K> getRegisteredKeysForKeyGroup(K key) {
-		checkArgument(localKeyGroupRange != null, "The operator has not been initialized.");
-		int keyGroupIdx = KeyGroupRangeAssignment.assignToKeyGroup(key, totalKeyGroups);
-		return getRegisteredKeysForKeyGroup(keyGroupIdx);
-	}
-
-	/**
-	 * Retrieve the set of keys for the requested key-group.
-	 *
-	 * @param keyGroupIdx the index of the key group we are interested in.
-	 * @return the set of keys for the key-group.
-	 */
-	private Set<K> getRegisteredKeysForKeyGroup(int keyGroupIdx) {
-		int localIdx = getIndexForKeyGroup(keyGroupIdx);
-		Set<K> keys = registeredKeysByKeyGroup[localIdx];
-		if (keys == null) {
-			keys = new HashSet<>();
-			registeredKeysByKeyGroup[localIdx] = keys;
-		}
-		return keys;
-	}
-
-	private Set<K> getDeletedKeysForKeyGroup(K key) {
-		checkArgument(localKeyGroupRange != null, "The operator has not been initialized.");
-		int keyGroupIdx = KeyGroupRangeAssignment.assignToKeyGroup(key, totalKeyGroups);
-		return getDeletedKeysForKeyGroup(keyGroupIdx);
-	}
-
-	private Set<K> getDeletedKeysForKeyGroup(int keyGroupIdx) {
-		int localIdx = getIndexForKeyGroup(keyGroupIdx);
-		Set<K> keys = deletedKeysByKeyGroup[localIdx];
-		if (keys == null) {
-			keys = new HashSet<>();
-			deletedKeysByKeyGroup[localIdx] = keys;
-		}
-		return keys;
-	}
-
-	/**
-	 * Computes the index of the requested key-group in the local datastructures.
-	 *
-	 * <p>Currently we assume that each task is assigned a continuous range of key-groups,
-	 * e.g. 1,2,3,4, and not 1,3,5. We leverage this to keep the different states
-	 * key-grouped in arrays instead of maps, where the offset for each key-group is
-	 * the key-group id (an int) minus the id of the first key-group in the local range.
-	 * This is for performance reasons.
-	 */
-	private int getIndexForKeyGroup(int keyGroupIdx) {
-		checkArgument(localKeyGroupRange.contains(keyGroupIdx),
-			"Key Group " + keyGroupIdx + " does not belong to the local range.");
-		return keyGroupIdx - localKeyGroupRangeStartIdx;
-	}
-
-	//////////////////				Fault Tolerance Methods				///////////////////
-
-	public void snapshotKeysForKeyGroup(DataOutputViewStreamWrapper stream, int keyGroupIdx) throws Exception {
-
-		// we cleanup also here to avoid checkpointing the deletion set
-		cleanupRegisteredKeys();
-
-		Set<K> keySet = getRegisteredKeysForKeyGroup(keyGroupIdx);
-		if (keySet != null) {
-			stream.writeInt(keySet.size());
-
-			InstantiationUtil.serializeObject(stream, keySerializer);
-			for (K key : keySet) {
-				keySerializer.serialize(key, stream);
-			}
-		} else {
-			stream.writeInt(0);
-		}
-	}
-
-	public void restoreKeysForKeyGroup(DataInputViewStreamWrapper stream, int keyGroupIdx,
-									ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException {
-
-		checkArgument(localKeyGroupRange.contains(keyGroupIdx),
-			"Key Group " + keyGroupIdx + " does not belong to the local range.");
-
-		int numKeys = stream.readInt();
-		if (numKeys > 0) {
-
-			TypeSerializer<K> tmpKeyDeserializer = InstantiationUtil.deserializeObject(stream, userCodeClassLoader);
-
-			if (keySerializer != null && !keySerializer.equals(tmpKeyDeserializer)) {
-				throw new IllegalArgumentException("Tried to restore keys " +
-					"for the watermark callback service with mismatching serializers.");
-			}
-
-			this.keySerializer = tmpKeyDeserializer;
-
-			Set<K> keys = getRegisteredKeysForKeyGroup(keyGroupIdx);
-			for (int i = 0; i < numKeys; i++) {
-				keys.add(keySerializer.deserialize(stream));
-			}
-		}
-	}
-
-	//////////////////				Testing Methods				///////////////////
-
-	@VisibleForTesting
-	public int numKeysForWatermarkCallback() {
-		int count = 0;
-		for (Set<K> keyGroup: registeredKeysByKeyGroup) {
-			if (keyGroup != null) {
-				count += keyGroup.size();
-			}
-		}
-		return count;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
index 1f68af9..774ab4a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
@@ -614,280 +614,6 @@ public class AbstractStreamOperatorTest {
 		verify(futureKeyedStateHandle).cancel(anyBoolean());
 	}
 
-	@Test
-	public void testWatermarkCallbackServiceScalingUp() throws Exception {
-		final int MAX_PARALLELISM = 10;
-
-		KeySelector<Tuple2<Integer, String>, Integer> keySelector = new TestKeySelector();
-
-		Tuple2<Integer, String> element1 = new Tuple2<>(7, "first");
-		Tuple2<Integer, String> element2 = new Tuple2<>(10, "start");
-
-		int keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element1), MAX_PARALLELISM);
-		assertEquals(1, keygroup);
-		assertEquals(0, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element2), MAX_PARALLELISM);
-		assertEquals(9, keygroup);
-		assertEquals(1, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		// now we start the test, we go from parallelism 1 to 2.
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness1 =
-			getTestHarness(MAX_PARALLELISM, 1, 0);
-		testHarness1.open();
-
-		testHarness1.processElement(new StreamRecord<>(element1));
-		testHarness1.processElement(new StreamRecord<>(element2));
-
-		assertEquals(0, testHarness1.getOutput().size());
-
-		// take a snapshot with some elements in internal sorting queue
-		OperatorStateHandles snapshot = testHarness1.snapshot(0, 0);
-		testHarness1.close();
-
-		// initialize two sub-tasks with the previously snapshotted state to simulate scaling up
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness2 =
-			getTestHarness(MAX_PARALLELISM, 2, 0);
-
-		testHarness2.setup();
-		testHarness2.initializeState(snapshot);
-		testHarness2.open();
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness3 =
-			getTestHarness(MAX_PARALLELISM, 2, 1);
-
-		testHarness3.setup();
-		testHarness3.initializeState(snapshot);
-		testHarness3.open();
-
-		testHarness2.processWatermark(new Watermark(10));
-		testHarness3.processWatermark(new Watermark(10));
-
-		assertEquals(2, testHarness2.getOutput().size());
-		verifyElement(testHarness2.getOutput().poll(), 7);
-		verifyWatermark(testHarness2.getOutput().poll(), 10);
-
-		assertEquals(2, testHarness3.getOutput().size());
-		verifyElement(testHarness3.getOutput().poll(), 10);
-		verifyWatermark(testHarness3.getOutput().poll(), 10);
-
-		testHarness1.close();
-		testHarness2.close();
-		testHarness3.close();
-	}
-
-	@Test
-	public void testWatermarkCallbackServiceScalingDown() throws Exception {
-		final int MAX_PARALLELISM = 10;
-
-		KeySelector<Tuple2<Integer, String>, Integer> keySelector = new TestKeySelector();
-
-		Tuple2<Integer, String> element1 = new Tuple2<>(7, "first");
-		Tuple2<Integer, String> element2 = new Tuple2<>(45, "start");
-		Tuple2<Integer, String> element3 = new Tuple2<>(90, "start");
-		Tuple2<Integer, String> element4 = new Tuple2<>(10, "start");
-
-		int keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element1), MAX_PARALLELISM);
-		assertEquals(1, keygroup);
-		assertEquals(0, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 3, keygroup));
-		assertEquals(0, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element2), MAX_PARALLELISM);
-		assertEquals(6, keygroup);
-		assertEquals(1, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 3, keygroup));
-		assertEquals(1, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element3), MAX_PARALLELISM);
-		assertEquals(2, keygroup);
-		assertEquals(0, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 3, keygroup));
-		assertEquals(0, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		keygroup = KeyGroupRangeAssignment.assignToKeyGroup(keySelector.getKey(element4), MAX_PARALLELISM);
-		assertEquals(9, keygroup);
-		assertEquals(2, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 3, keygroup));
-		assertEquals(1, KeyGroupRangeAssignment.computeOperatorIndexForKeyGroup(MAX_PARALLELISM, 2, keygroup));
-
-		// starting the test, we will go from parallelism of 3 to parallelism of 2
-
-		// first operator
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness1 =
-			getTestHarness(MAX_PARALLELISM, 3, 0);
-		testHarness1.open();
-
-		// second operator
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness2 =
-			getTestHarness(MAX_PARALLELISM, 3, 1);
-		testHarness2.open();
-
-		// third operator
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness3 =
-			getTestHarness(MAX_PARALLELISM, 3, 2);
-		testHarness3.open();
-
-		testHarness1.processWatermark(Long.MIN_VALUE);
-		testHarness2.processWatermark(Long.MIN_VALUE);
-		testHarness3.processWatermark(Long.MIN_VALUE);
-
-		testHarness1.processElement(new StreamRecord<>(element1));
-		testHarness1.processElement(new StreamRecord<>(element3));
-
-		testHarness2.processElement(new StreamRecord<>(element2));
-		testHarness3.processElement(new StreamRecord<>(element4));
-
-		// so far we only have the initial watermark
-		assertEquals(1, testHarness1.getOutput().size());
-		verifyWatermark(testHarness1.getOutput().poll(), Long.MIN_VALUE);
-
-		assertEquals(1, testHarness2.getOutput().size());
-		verifyWatermark(testHarness2.getOutput().poll(), Long.MIN_VALUE);
-
-		assertEquals(1, testHarness3.getOutput().size());
-		verifyWatermark(testHarness3.getOutput().poll(), Long.MIN_VALUE);
-
-		// we take a snapshot and make it look as a single operator
-		// this will be the initial state of all downstream tasks.
-		OperatorStateHandles snapshot = AbstractStreamOperatorTestHarness.repackageState(
-			testHarness2.snapshot(0, 0),
-			testHarness1.snapshot(0, 0),
-			testHarness3.snapshot(0, 0)
-		);
-
-		// first new operator
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness4 =
-			getTestHarness(MAX_PARALLELISM, 2, 0);
-		testHarness4.setup();
-		testHarness4.initializeState(snapshot);
-		testHarness4.open();
-
-		// second new operator
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness5 =
-			getTestHarness(MAX_PARALLELISM, 2, 1);
-		testHarness5.setup();
-		testHarness5.initializeState(snapshot);
-		testHarness5.open();
-
-		testHarness4.processWatermark(10);
-		testHarness5.processWatermark(10);
-
-		assertEquals(3, testHarness4.getOutput().size());
-		verifyElement(testHarness4.getOutput().poll(), 7);
-		verifyElement(testHarness4.getOutput().poll(), 90);
-		verifyWatermark(testHarness4.getOutput().poll(), 10);
-
-		assertEquals(3, testHarness5.getOutput().size());
-		verifyElement(testHarness5.getOutput().poll(), 45);
-		verifyElement(testHarness5.getOutput().poll(), 10);
-		verifyWatermark(testHarness5.getOutput().poll(), 10);
-
-		testHarness1.close();
-		testHarness2.close();
-		testHarness3.close();
-		testHarness4.close();
-		testHarness5.close();
-	}
-
-	@Test
-	public void testWatermarkCallbackServiceKeyDeletion() throws Exception {
-		final int MAX_PARALLELISM = 10;
-
-		Tuple2<Integer, String> element1 = new Tuple2<>(7, "start");
-		Tuple2<Integer, String> element2 = new Tuple2<>(45, "start");
-		Tuple2<Integer, String> element3 = new Tuple2<>(90, "start");
-
-		TestOperatorWithDeletingKeyCallback op = new TestOperatorWithDeletingKeyCallback(45);
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> testHarness1 =
-			new KeyedOneInputStreamOperatorTestHarness<>(
-				op,
-				new TestKeySelector(),
-				BasicTypeInfo.INT_TYPE_INFO,
-				MAX_PARALLELISM,
-				1,
-				0);
-		testHarness1.open();
-
-		testHarness1.processElement(new StreamRecord<>(element1));
-		testHarness1.processElement(new StreamRecord<>(element2));
-
-		testHarness1.processWatermark(10L);
-
-		assertEquals(3L, testHarness1.getOutput().size());
-		verifyElement(testHarness1.getOutput().poll(), 7);
-		verifyElement(testHarness1.getOutput().poll(), 45);
-		verifyWatermark(testHarness1.getOutput().poll(), 10);
-
-		testHarness1.processElement(new StreamRecord<>(element3));
-		testHarness1.processWatermark(20L);
-
-		// because at the first watermark the operator removed key 45
-		assertEquals(3L, testHarness1.getOutput().size());
-		verifyElement(testHarness1.getOutput().poll(), 7);
-		verifyElement(testHarness1.getOutput().poll(), 90);
-		verifyWatermark(testHarness1.getOutput().poll(), 20);
-
-		testHarness1.processWatermark(25L);
-
-		verifyElement(testHarness1.getOutput().poll(), 7);
-		verifyElement(testHarness1.getOutput().poll(), 90);
-		verifyWatermark(testHarness1.getOutput().poll(), 25);
-
-		// unregister key and then fail
-		op.unregisterKey(90);
-
-		// take a snapshot with some elements in internal sorting queue
-		OperatorStateHandles snapshot = testHarness1.snapshot(0, 0);
-		testHarness1.close();
-
-		testHarness1 = new KeyedOneInputStreamOperatorTestHarness<>(
-				new TestOperatorWithDeletingKeyCallback(45),
-				new TestKeySelector(),
-				BasicTypeInfo.INT_TYPE_INFO,
-				MAX_PARALLELISM,
-				1,
-				0);
-		testHarness1.setup();
-		testHarness1.initializeState(snapshot);
-		testHarness1.open();
-
-		testHarness1.processWatermark(30L);
-
-		assertEquals(2L, testHarness1.getOutput().size());
-		verifyElement(testHarness1.getOutput().poll(), 7);
-		verifyWatermark(testHarness1.getOutput().poll(), 30);
-
-		testHarness1.close();
-	}
-
-	private KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, String>, Integer> getTestHarness(
-			int maxParallelism, int noOfTasks, int taskIdx) throws Exception {
-
-		return new KeyedOneInputStreamOperatorTestHarness<>(
-			new TestOperatorWithCallback(),
-			new TestKeySelector(),
-			BasicTypeInfo.INT_TYPE_INFO,
-			maxParallelism,
-			noOfTasks, /* num subtasks */
-			taskIdx /* subtask index */);
-	}
-
-	private void verifyWatermark(Object outputObject, long timestamp) {
-		Assert.assertTrue(outputObject instanceof Watermark);
-		assertEquals(timestamp, ((Watermark) outputObject).getTimestamp());
-	}
-
-	private void verifyElement(Object outputObject, int expected) {
-		Assert.assertTrue(outputObject instanceof StreamRecord);
-
-		StreamRecord<?> resultRecord = (StreamRecord<?>) outputObject;
-		Assert.assertTrue(resultRecord.getValue() instanceof Integer);
-
-		@SuppressWarnings("unchecked")
-		int actual = (Integer) resultRecord.getValue();
-		assertEquals(expected, actual);
-	}
-
 	/**
 	 * Extracts the result values form the test harness and clear the output queue.
 	 */
@@ -913,78 +639,6 @@ public class AbstractStreamOperatorTest {
 		}
 	}
 
-	private static class TestOperatorWithCallback
-			extends AbstractStreamOperator<Integer>
-			implements OneInputStreamOperator<Tuple2<Integer, String>, Integer> {
-
-		private static final long serialVersionUID = 9215057823264582305L;
-
-		@Override
-		public void open() throws Exception {
-			super.open();
-
-			InternalWatermarkCallbackService<Integer> callbackService = getInternalWatermarkCallbackService();
-
-			callbackService.setWatermarkCallback(new OnWatermarkCallback<Integer>() {
-
-				@Override
-				public void onWatermark(Integer integer, Watermark watermark) throws IOException {
-					output.collect(new StreamRecord<>(integer));
-				}
-			}, IntSerializer.INSTANCE);
-		}
-
-		@Override
-		public void processElement(StreamRecord<Tuple2<Integer, String>> element) throws Exception {
-			getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(element.getValue().f0);
-		}
-	}
-
-	private static class TestOperatorWithDeletingKeyCallback
-			extends AbstractStreamOperator<Integer>
-			implements OneInputStreamOperator<Tuple2<Integer, String>, Integer> {
-
-		private static final long serialVersionUID = 9215057823264582305L;
-
-		private final int keyToDelete;
-
-		public TestOperatorWithDeletingKeyCallback(int keyToDelete) {
-			this.keyToDelete = keyToDelete;
-		}
-
-		@Override
-		public void open() throws Exception {
-			super.open();
-
-			InternalWatermarkCallbackService<Integer> callbackService = getInternalWatermarkCallbackService();
-
-			callbackService.setWatermarkCallback(new OnWatermarkCallback<Integer>() {
-
-				@Override
-				public void onWatermark(Integer integer, Watermark watermark) throws IOException {
-
-					// this is to simulate the case where we may have a concurrent modification
-					// exception as we iterate over the list of registered keys and we concurrently
-					// delete the key.
-
-					if (integer.equals(keyToDelete)) {
-						getInternalWatermarkCallbackService().unregisterKeyFromWatermarkCallback(integer);
-					}
-					output.collect(new StreamRecord<>(integer));
-				}
-			}, IntSerializer.INSTANCE);
-		}
-
-		@Override
-		public void processElement(StreamRecord<Tuple2<Integer, String>> element) throws Exception {
-			getInternalWatermarkCallbackService().registerKeyForWatermarkCallback(element.getValue().f0);
-		}
-
-		public void unregisterKey(int key) {
-			getInternalWatermarkCallbackService().unregisterKeyFromWatermarkCallback(key);
-		}
-	}
-
 	/**
 	 * Testing operator that can respond to commands by either setting/deleting state, emitting
 	 * state or setting timers.

http://git-wip-us.apache.org/repos/asf/flink/blob/6f8b3c61/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 a2a4f79..4a9463a 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
@@ -610,15 +610,6 @@ public class AbstractStreamOperatorTestHarness<OUT> {
 		}
 	}
 
-	@VisibleForTesting
-	public int numKeysForWatermarkCallback() {
-		if (operator instanceof AbstractStreamOperator) {
-			return ((AbstractStreamOperator) operator).numKeysForWatermarkCallback();
-		} else {
-			throw new UnsupportedOperationException();
-		}
-	}
-
 	private class MockOutput implements Output<StreamRecord<OUT>> {
 
 		private TypeSerializer<OUT> outputSerializer;


[07/13] 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/9ed98f2e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9ed98f2e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9ed98f2e

Branch: refs/heads/master
Commit: 9ed98f2e5a32fb14de03b9a8ea1dd45851cc3a7e
Parents: 1882c90
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Apr 20 10:46:10 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 16:24:26 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/9ed98f2e/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 301fc72..c5672a2 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
@@ -155,15 +155,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) {
@@ -174,24 +169,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/9ed98f2e/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/9ed98f2e/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/9ed98f2e/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/9ed98f2e/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/9ed98f2e/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


[10/13] 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/2c6377f2
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2c6377f2
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2c6377f2

Branch: refs/heads/master
Commit: 2c6377f257cafcbaea3a5bf33dd27852ed05afec
Parents: 84eea72
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 16:25:57 2017 +0200

----------------------------------------------------------------------
 .../WindowOperatorFrom11MigrationTest.java      |  896 ++++++++++++++++
 .../WindowOperatorFrom12MigrationTest.java      | 1014 ++++++++++++++++++
 .../windowing/WindowOperatorMigrationTest.java  |  896 ----------------
 ...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, 1910 insertions(+), 896 deletions(-)
----------------------------------------------------------------------


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

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


[02/13] 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/44e472f6
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/44e472f6
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/44e472f6

Branch: refs/heads/master
Commit: 44e472f6cffd75213c3b7373bd622b734f89f0b0
Parents: 2779197
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 16:24:26 2017 +0200

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


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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/44e472f6/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 e271a21..0000000
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingMigrationTest.java
+++ /dev/null
@@ -1,402 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.hdfstests;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
-import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
-import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-
-public class 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(ConfigConstants.DEFAULT_CHARSET));
-		}
-		stream.close();
-
-		hdfs.rename(tmp, file);
-
-		Assert.assertTrue("No result file present", hdfs.exists(file));
-		return new Tuple2<>(file, str.toString());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/44e472f6/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/44e472f6/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


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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/2c6377f2/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/2c6377f2/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/2c6377f2/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/2c6377f2/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/2c6377f2/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/2c6377f2/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/2c6377f2/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/2c6377f2/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


[06/13] 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/1882c905
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1882c905
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1882c905

Branch: refs/heads/master
Commit: 1882c90505b0d25775b969cc025c8a3087b82f37
Parents: 6f8b3c6
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 16:24:26 2017 +0200

----------------------------------------------------------------------
 .../flink/client/program/ClusterClient.java     | 32 +++++++++++++++++++
 .../utils/SavepointMigrationTestBase.java       | 33 +++++++++++++++++++-
 2 files changed, 64 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1882c905/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/1882c905/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 fced68c..301fc72 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
@@ -25,14 +25,17 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.JobSubmissionResult;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.configuration.CoreOptions;
 import org.apache.flink.runtime.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;
@@ -49,6 +52,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;
@@ -207,11 +211,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) {


[05/13] 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/2779197f
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2779197f
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2779197f

Branch: refs/heads/master
Commit: 2779197f237446e3bff4e9e15f90c24d721c8ab4
Parents: 9ed98f2
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 16:24:26 2017 +0200

----------------------------------------------------------------------
 .../savepoint/SavepointV1Serializer.java        |  27 ++--
 .../streaming/util/OperatorSnapshotUtil.java    | 156 +++++++++++++++++++
 2 files changed, 174 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2779197f/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 aaa8cdd..f67d54c 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;
@@ -43,12 +45,13 @@ import java.util.Map;
 
 /**
  * Deserializer for checkpoints written in format {@code 1} (Flink 1.2.x format)
- * 
+ *
  * <p>In contrast to the previous versions, this serializer makes sure that no Java
- * serialization is used for serialization. Therefore, we don't rely on any involved 
+ * serialization is used for serialization. Therefore, we don't rely on any involved
  * classes to stay the same.
  */
-class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
+@Internal
+public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 
 	private static final byte NULL_HANDLE = 0;
 	private static final byte BYTE_STREAM_STATE_HANDLE = 1;
@@ -210,7 +213,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 				keyedStateStream);
 	}
 
-	private static void serializeKeyedStateHandle(
+	@VisibleForTesting
+	public static void serializeKeyedStateHandle(
 			KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle == null) {
@@ -230,7 +234,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		}
 	}
 
-	private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException {
+	@VisibleForTesting
+	public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException {
 		final int type = dis.readByte();
 		if (NULL_HANDLE == type) {
 			return null;
@@ -251,7 +256,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		}
 	}
 
-	private static void serializeOperatorStateHandle(
+	@VisibleForTesting
+	public static void serializeOperatorStateHandle(
 			OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle != null) {
@@ -279,7 +285,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		}
 	}
 
-	private static OperatorStateHandle deserializeOperatorStateHandle(
+	@VisibleForTesting
+	public static OperatorStateHandle deserializeOperatorStateHandle(
 			DataInputStream dis) throws IOException {
 
 		final int type = dis.readByte();
@@ -310,7 +317,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		}
 	}
 
-	private static void serializeStreamStateHandle(
+	@VisibleForTesting
+	public static void serializeStreamStateHandle(
 			StreamStateHandle stateHandle, DataOutputStream dos) throws IOException {
 
 		if (stateHandle == null) {
@@ -337,7 +345,8 @@ class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		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/2779197f/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..92a9452
--- /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.KeyedStateHandle;
+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<KeyedStateHandle> rawKeyedState = state.getRawKeyedState();
+		if (rawKeyedState != null) {
+			dos.writeInt(rawKeyedState.size());
+			for (KeyedStateHandle keyedStateHandle : rawKeyedState) {
+				SavepointV1Serializer.serializeKeyedStateHandle(keyedStateHandle, dos);
+			}
+		} else {
+			// this means no operator states, not even an empty list
+			dos.writeInt(-1);
+		}
+
+		Collection<KeyedStateHandle> managedKeyedState = state.getManagedKeyedState();
+		if (managedKeyedState != null) {
+			dos.writeInt(managedKeyedState.size());
+			for (KeyedStateHandle keyedStateHandle : managedKeyedState) {
+				SavepointV1Serializer.serializeKeyedStateHandle(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<KeyedStateHandle> rawKeyedState = null;
+		int numRawKeyedStates = dis.readInt();
+		if (numRawKeyedStates >= 0) {
+			rawKeyedState = new ArrayList<>();
+			for (int i = 0; i < numRawKeyedStates; i++) {
+				KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle(
+						dis);
+				rawKeyedState.add(keyedState);
+			}
+		}
+
+		List<KeyedStateHandle> managedKeyedState = null;
+		int numManagedKeyedStates = dis.readInt();
+		if (numManagedKeyedStates >= 0) {
+			managedKeyedState = new ArrayList<>();
+			for (int i = 0; i < numManagedKeyedStates; i++) {
+				KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle(
+						dis);
+				managedKeyedState.add(keyedState);
+			}
+		}
+
+		return new OperatorStateHandles(index, legacyState, managedKeyedState, rawKeyedState, managedOperatorState, rawOperatorState);
+	}
+}


[08/13] 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/fb7793f0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fb7793f0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fb7793f0

Branch: refs/heads/master
Commit: fb7793f033cfa0d6d77ef25a6c518a5a203ebb82
Parents: 2c6377f
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 16:25:57 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/fb7793f0/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/fb7793f0/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/fb7793f0/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


[11/13] 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/e40f2e18
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e40f2e18
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e40f2e18

Branch: refs/heads/master
Commit: e40f2e184c6e57d4346312f969a64727389e92fa
Parents: 0ecb5d0
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 16:25:57 2017 +0200

----------------------------------------------------------------------
 ...inkKafkaConsumerBaseFrom11MigrationTest.java | 347 +++++++++++++++++++
 ...inkKafkaConsumerBaseFrom12MigrationTest.java | 338 ++++++++++++++++++
 .../FlinkKafkaConsumerBaseMigrationTest.java    | 347 -------------------
 ...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, 685 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/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/e40f2e18/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